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/06/12 15:45:08 UTC

[01/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Repository: incubator-distributedlog
Updated Branches:
  refs/heads/master 52c0eef87 -> c44e0278e


http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
deleted file mode 100644
index 431bfa4..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream.limiter;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.ConcurrentConstConfiguration;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.limiter.ChainedRequestLimiter;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import org.apache.distributedlog.limiter.GuavaRateLimiter;
-import org.apache.distributedlog.limiter.RateLimiter;
-import org.apache.distributedlog.limiter.RequestLimiter;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServiceRequestLimiter}.
- */
-public class TestServiceRequestLimiter {
-
-    /**
-     * Mock Request.
-     */
-    class MockRequest {
-        int size;
-        MockRequest() {
-            this(1);
-        }
-        MockRequest(int size) {
-            this.size = size;
-        }
-        int getSize() {
-            return size;
-        }
-    }
-
-    /**
-     * Mock request limiter.
-     */
-    class MockRequestLimiter implements RequestLimiter<MockRequest> {
-        public void apply(MockRequest request) {
-        }
-    }
-
-    /**
-     * Counter based limiter.
-     */
-    static class CounterLimiter implements RateLimiter {
-        final int limit;
-        int count;
-
-        public CounterLimiter(int limit) {
-            this.limit = limit;
-            this.count = 0;
-        }
-
-        @Override
-        public boolean acquire(int permits) {
-            if (++count > limit) {
-                return false;
-            }
-            return true;
-        }
-    }
-
-    /**
-     * Mock hard request limiter.
-     */
-    class MockHardRequestLimiter implements RequestLimiter<MockRequest> {
-
-        RequestLimiter<MockRequest> limiter;
-        int limitHitCount;
-
-        MockHardRequestLimiter(int limit) {
-            this(GuavaRateLimiter.of(limit));
-        }
-
-        MockHardRequestLimiter(RateLimiter limiter) {
-            this.limiter = new ComposableRequestLimiter<MockRequest>(
-                limiter,
-                new OverlimitFunction<MockRequest>() {
-                    public void apply(MockRequest request) throws OverCapacityException {
-                        limitHitCount++;
-                        throw new OverCapacityException("Limit exceeded");
-                    }
-                },
-                new CostFunction<MockRequest>() {
-                    public int apply(MockRequest request) {
-                        return request.getSize();
-                    }
-                },
-                NullStatsLogger.INSTANCE);
-        }
-
-        @Override
-        public void apply(MockRequest op) throws OverCapacityException {
-            limiter.apply(op);
-        }
-
-        public int getLimitHitCount() {
-            return limitHitCount;
-        }
-    }
-
-    /**
-     * Mock soft request limiter.
-     */
-    class MockSoftRequestLimiter implements RequestLimiter<MockRequest> {
-
-        RequestLimiter<MockRequest> limiter;
-        int limitHitCount;
-
-        MockSoftRequestLimiter(int limit) {
-            this(GuavaRateLimiter.of(limit));
-        }
-
-        MockSoftRequestLimiter(RateLimiter limiter) {
-            this.limiter = new ComposableRequestLimiter<MockRequest>(
-                limiter,
-                new OverlimitFunction<MockRequest>() {
-                    public void apply(MockRequest request) throws OverCapacityException {
-                        limitHitCount++;
-                    }
-                },
-                new CostFunction<MockRequest>() {
-                    public int apply(MockRequest request) {
-                        return request.getSize();
-                    }
-                },
-                NullStatsLogger.INSTANCE);
-        }
-
-        @Override
-        public void apply(MockRequest op) throws OverCapacityException {
-            limiter.apply(op);
-        }
-
-        public int getLimitHitCount() {
-            return limitHitCount;
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testDynamicLimiter() throws Exception {
-        final AtomicInteger id = new AtomicInteger(0);
-        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
-        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
-                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
-            @Override
-            public RequestLimiter<MockRequest> build() {
-                id.getAndIncrement();
-                return new MockRequestLimiter();
-            }
-        };
-        limiter.initialize();
-        assertEquals(1, id.get());
-        dynConf.setProperty("test1", 1);
-        assertEquals(2, id.get());
-        dynConf.setProperty("test2", 2);
-        assertEquals(3, id.get());
-    }
-
-    @Test(timeout = 60000)
-    public void testDynamicLimiterWithDisabledFeature() throws Exception {
-        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
-        final MockSoftRequestLimiter rateLimiter = new MockSoftRequestLimiter(0);
-        final SettableFeature disabledFeature = new SettableFeature("", 0);
-        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
-                dynConf, NullStatsLogger.INSTANCE, disabledFeature) {
-            @Override
-            public RequestLimiter<MockRequest> build() {
-                return rateLimiter;
-            }
-        };
-        limiter.initialize();
-        assertEquals(0, rateLimiter.getLimitHitCount());
-
-        // Not disabled, rate limiter was invoked
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        assertEquals(1, rateLimiter.getLimitHitCount());
-
-        // Disabled, rate limiter not invoked
-        disabledFeature.set(1);
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        assertEquals(1, rateLimiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testDynamicLimiterWithException() throws Exception {
-        final AtomicInteger id = new AtomicInteger(0);
-        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
-        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
-                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
-            @Override
-            public RequestLimiter<MockRequest> build() {
-                if (id.incrementAndGet() >= 2) {
-                    throw new RuntimeException("exception in dynamic limiter build()");
-                }
-                return new MockRequestLimiter();
-            }
-        };
-        limiter.initialize();
-        assertEquals(1, id.get());
-        try {
-            dynConf.setProperty("test1", 1);
-            fail("should have thrown on config failure");
-        } catch (RuntimeException ex) {
-        }
-        assertEquals(2, id.get());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceRequestLimiter() throws Exception {
-        MockHardRequestLimiter limiter = new MockHardRequestLimiter(new CounterLimiter(1));
-        limiter.apply(new MockRequest());
-        try {
-            limiter.apply(new MockRequest());
-        } catch (OverCapacityException ex) {
-        }
-        assertEquals(1, limiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceRequestLimiterWithDefaultRate() throws Exception {
-        MockHardRequestLimiter limiter = new MockHardRequestLimiter(-1);
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        assertEquals(0, limiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceRequestLimiterWithZeroRate() throws Exception {
-        MockHardRequestLimiter limiter = new MockHardRequestLimiter(0);
-        try {
-            limiter.apply(new MockRequest(1));
-            fail("should have failed with overcap");
-        } catch (OverCapacityException ex) {
-        }
-        assertEquals(1, limiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testChainedServiceRequestLimiter() throws Exception {
-        MockSoftRequestLimiter softLimiter = new MockSoftRequestLimiter(new CounterLimiter(1));
-        MockHardRequestLimiter hardLimiter = new MockHardRequestLimiter(new CounterLimiter(3));
-
-        RequestLimiter<MockRequest> limiter =
-                new ChainedRequestLimiter.Builder<MockRequest>()
-                .addLimiter(softLimiter)
-                .addLimiter(hardLimiter)
-                .build();
-
-        assertEquals(0, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        limiter.apply(new MockRequest());
-        assertEquals(0, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        limiter.apply(new MockRequest());
-        assertEquals(1, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        limiter.apply(new MockRequest());
-        assertEquals(2, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        try {
-            limiter.apply(new MockRequest());
-        } catch (OverCapacityException ex) {
-        }
-        assertEquals(3, softLimiter.getLimitHitCount());
-        assertEquals(1, hardLimiter.getLimitHitCount());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
deleted file mode 100644
index 15a0753..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.streamset;
-
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-/**
- * Test Cases for {@link DelimiterStreamPartitionConverter}.
- */
-public class TestDelimiterStreamPartitionConverter {
-
-    @Test(timeout = 20000)
-    public void testNormalStream() throws Exception {
-        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
-        assertEquals(new Partition("distributedlog-smoketest", 1),
-                converter.convert("distributedlog-smoketest_1"));
-        assertEquals(new Partition("distributedlog-smoketest-", 1),
-                converter.convert("distributedlog-smoketest-_1"));
-        assertEquals(new Partition("distributedlog-smoketest", 1),
-                converter.convert("distributedlog-smoketest_000001"));
-    }
-
-    private void assertIdentify(String streamName, StreamPartitionConverter converter) {
-        assertEquals(new Partition(streamName, 0), converter.convert(streamName));
-    }
-
-    @Test(timeout = 20000)
-    public void testUnknownStream() throws Exception {
-        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
-        assertIdentify("test1", converter);
-        assertIdentify("test1-000001", converter);
-        assertIdentify("test1_test1_000001", converter);
-        assertIdentify("test1_test1", converter);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
deleted file mode 100644
index 1a5d8d3..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.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 org.apache.distributedlog.service.streamset;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-/**
- * Test Case for {@link IdentityStreamPartitionConverter}.
- */
-public class TestIdentityStreamPartitionConverter {
-
-    @Test(timeout = 20000)
-    public void testIdentityConverter() {
-        String streamName = "test-identity-converter";
-
-        IdentityStreamPartitionConverter converter =
-                new IdentityStreamPartitionConverter();
-
-        Partition p0 = converter.convert(streamName);
-        assertEquals(new Partition(streamName, 0), p0);
-
-        Partition p1 = converter.convert(streamName);
-        assertTrue(p0 == p1);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java
deleted file mode 100644
index b6e5ff3..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.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 org.apache.distributedlog.service.streamset;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-/**
- * Test {@link PartitionMap}.
- */
-public class TestPartitionMap {
-
-    @Test(timeout = 20000)
-    public void testAddPartitionNegativeMax() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 20; i++) {
-            assertTrue(map.addPartition(new Partition("test", i), -1));
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void testAddPartitionMultipleTimes() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 20; i++) {
-            assertTrue(map.addPartition(new Partition("test", 0), 3));
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void testAddPartition() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 3; i++) {
-            assertTrue(map.addPartition(new Partition("test", i), 3));
-        }
-        for (int i = 3; i < 20; i++) {
-            assertFalse(map.addPartition(new Partition("test", i), 3));
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void testRemovePartition() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 3; i++) {
-            assertTrue(map.addPartition(new Partition("test", i), 3));
-        }
-        assertFalse(map.addPartition(new Partition("test", 3), 3));
-        assertFalse(map.removePartition(new Partition("test", 3)));
-        assertTrue(map.removePartition(new Partition("test", 0)));
-        assertTrue(map.addPartition(new Partition("test", 3), 3));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java
deleted file mode 100644
index 2853df1..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.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 org.apache.distributedlog.service.utils;
-
-import static org.junit.Assert.assertEquals;
-
-import java.net.InetAddress;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServerUtils}.
- */
-public class TestServerUtils {
-
-    @Test(timeout = 6000)
-    public void testGetLedgerAllocatorPoolName() throws Exception {
-        int region = 123;
-        int shard = 999;
-        String hostname = InetAddress.getLocalHost().getHostAddress();
-        assertEquals("allocator_0123_0000000999",
-            ServerUtils.getLedgerAllocatorPoolName(region, shard, false));
-        assertEquals("allocator_0123_" + hostname,
-            ServerUtils.getLedgerAllocatorPoolName(region, shard, true));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/resources/log4j.properties b/distributedlog-service/src/test/resources/log4j.properties
deleted file mode 100644
index 3e51059..0000000
--- a/distributedlog-service/src/test/resources/log4j.properties
+++ /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.
-# */
-
-#
-# DisributedLog Logging Configuration
-#
-
-# Example with rolling log file
-log4j.rootLogger=INFO, CONSOLE
-
-#disable zookeeper logging
-log4j.logger.org.apache.zookeeper=OFF
-#Set the bookkeeper level to warning
-log4j.logger.org.apache.bookkeeper=INFO
-
-log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
-log4j.appender.CONSOLE.Threshold=INFO
-log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
-log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-# Add ROLLINGFILE to rootLogger to get log file output
-#    Log DEBUG level and above messages to a log file
-#log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
-#log4j.appender.ROLLINGFILE.Threshold=INFO
-#log4j.appender.ROLLINGFILE.File=distributedlog.log
-#log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
-#log4j.appender.ROLLINGFILE.DatePattern='.'yyyy-MM-dd-HH-mm
-#log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-log4j.appender.R=org.apache.log4j.RollingFileAppender
-log4j.appender.R.Threshold=TRACE
-log4j.appender.R.File=target/error.log
-log4j.appender.R.MaxFileSize=200MB
-log4j.appender.R.MaxBackupIndex=7
-log4j.appender.R.layout=org.apache.log4j.PatternLayout
-log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-tutorials/distributedlog-basic/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-tutorials/distributedlog-basic/pom.xml b/distributedlog-tutorials/distributedlog-basic/pom.xml
index 34ce4f7..fde8e56 100644
--- a/distributedlog-tutorials/distributedlog-basic/pom.xml
+++ b/distributedlog-tutorials/distributedlog-basic/pom.xml
@@ -39,7 +39,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-client</artifactId>
+      <artifactId>distributedlog-proxy-client</artifactId>
       <version>${project.parent.version}</version>
       <exclusions>
         <exclusion>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-tutorials/distributedlog-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/pom.xml b/distributedlog-tutorials/distributedlog-mapreduce/pom.xml
index d477221..7a6b67a 100644
--- a/distributedlog-tutorials/distributedlog-mapreduce/pom.xml
+++ b/distributedlog-tutorials/distributedlog-mapreduce/pom.xml
@@ -39,7 +39,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-client</artifactId>
+      <artifactId>distributedlog-proxy-client</artifactId>
       <version>${project.parent.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-tutorials/distributedlog-messaging/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-tutorials/distributedlog-messaging/pom.xml b/distributedlog-tutorials/distributedlog-messaging/pom.xml
index cb6a1a7..97af77b 100644
--- a/distributedlog-tutorials/distributedlog-messaging/pom.xml
+++ b/distributedlog-tutorials/distributedlog-messaging/pom.xml
@@ -39,7 +39,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-client</artifactId>
+      <artifactId>distributedlog-proxy-client</artifactId>
       <version>${project.parent.version}</version>
       <exclusions>
         <exclusion>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d10eb85..dafda4a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -85,8 +85,9 @@
     <module>distributedlog-build-tools</module>
     <module>distributedlog-protocol</module>
     <module>distributedlog-core</module>
-    <module>distributedlog-client</module>
-    <module>distributedlog-service</module>
+    <module>distributedlog-proxy-protocol</module>
+    <module>distributedlog-proxy-client</module>
+    <module>distributedlog-proxy-server</module>
     <module>distributedlog-benchmark</module>
     <module>distributedlog-tutorials</module>
   </modules>
@@ -230,6 +231,8 @@
             <exclude>**/**.md</exclude>
             <exclude>scripts/dev/reviewers</exclude>
             <exclude>src/main/resources/DISCLAIMER.bin.txt</exclude>            
+            <exclude>**/dependency-reduced-pom.xml</exclude>
+            <exclude>**/org/apache/distributedlog/thrift/*</exclude>
           </excludes>
         </configuration>
       </plugin>


[18/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ClientUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ClientUtils.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ClientUtils.java
new file mode 100644
index 0000000..96bc338
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ClientUtils.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.service;
+
+import org.apache.distributedlog.client.DistributedLogClientImpl;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * DistributedLog Client Related Utils.
+ */
+public class ClientUtils {
+
+    public static Pair<DistributedLogClient, MonitorServiceClient> buildClient(DistributedLogClientBuilder builder) {
+        DistributedLogClientImpl clientImpl = builder.buildClient();
+        return Pair.of((DistributedLogClient) clientImpl, (MonitorServiceClient) clientImpl);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
new file mode 100644
index 0000000..9cc085d
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
@@ -0,0 +1,352 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.DistributedLogConfiguration;
+import org.apache.distributedlog.LocalDLMEmulator;
+import org.apache.distributedlog.client.routing.SingleHostRoutingService;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import com.twitter.finagle.builder.Server;
+import java.io.File;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DistributedLog Cluster is an emulator to run distributedlog components.
+ */
+public class DistributedLogCluster {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DistributedLogCluster.class);
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder to build distributedlog cluster.
+     */
+    public static class Builder {
+
+        int numBookies = 3;
+        boolean shouldStartZK = true;
+        String zkHost = "127.0.0.1";
+        int zkPort = 0;
+        boolean shouldStartProxy = true;
+        int proxyPort = 7000;
+        boolean thriftmux = false;
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration()
+                .setLockTimeout(10)
+                .setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true);
+        ServerConfiguration bkConf = new ServerConfiguration();
+
+        private Builder() {}
+
+        /**
+         * How many bookies to run. By default is 3.
+         *
+         * @return builder
+         */
+        public Builder numBookies(int numBookies) {
+            this.numBookies = numBookies;
+            return this;
+        }
+
+        /**
+         * Whether to start zookeeper? By default is true.
+         *
+         * @param startZK
+         *          flag to start zookeeper?
+         * @return builder
+         */
+        public Builder shouldStartZK(boolean startZK) {
+            this.shouldStartZK = startZK;
+            return this;
+        }
+
+        /**
+         * ZooKeeper server to run. By default it runs locally on '127.0.0.1'.
+         *
+         * @param zkServers
+         *          zk servers
+         * @return builder
+         */
+        public Builder zkServers(String zkServers) {
+            this.zkHost = zkServers;
+            return this;
+        }
+
+        /**
+         * ZooKeeper server port to listen on. By default it listens on 2181.
+         *
+         * @param zkPort
+         *          zookeeper server port.
+         * @return builder.
+         */
+        public Builder zkPort(int zkPort) {
+            this.zkPort = zkPort;
+            return this;
+        }
+
+        /**
+         * Whether to start proxy or not. By default is true.
+         *
+         * @param startProxy
+         *          whether to start proxy or not.
+         * @return builder
+         */
+        public Builder shouldStartProxy(boolean startProxy) {
+            this.shouldStartProxy = startProxy;
+            return this;
+        }
+
+        /**
+         * Port that proxy server to listen on. By default is 7000.
+         *
+         * @param proxyPort
+         *          port that proxy server to listen on.
+         * @return builder
+         */
+        public Builder proxyPort(int proxyPort) {
+            this.proxyPort = proxyPort;
+            return this;
+        }
+
+        /**
+         * Set the distributedlog configuration.
+         *
+         * @param dlConf
+         *          distributedlog configuration
+         * @return builder
+         */
+        public Builder dlConf(DistributedLogConfiguration dlConf) {
+            this.dlConf = dlConf;
+            return this;
+        }
+
+        /**
+         * Set the Bookkeeper server configuration.
+         *
+         * @param bkConf
+         *          bookkeeper server configuration
+         * @return builder
+         */
+        public Builder bkConf(ServerConfiguration bkConf) {
+            this.bkConf = bkConf;
+            return this;
+        }
+
+        /**
+         * Enable thriftmux for the dl server.
+         *
+         * @param enabled flag to enable thriftmux
+         * @return builder
+         */
+        public Builder thriftmux(boolean enabled) {
+            this.thriftmux = enabled;
+            return this;
+        }
+
+        public DistributedLogCluster build() throws Exception {
+            // build the cluster
+            return new DistributedLogCluster(
+                dlConf,
+                bkConf,
+                numBookies,
+                shouldStartZK,
+                zkHost,
+                zkPort,
+                shouldStartProxy,
+                proxyPort,
+                thriftmux);
+        }
+    }
+
+    /**
+     * Run a distributedlog proxy server.
+     */
+    public static class DLServer {
+
+        static final int MAX_RETRIES = 20;
+        static final int MIN_PORT = 1025;
+        static final int MAX_PORT = 65535;
+
+        int proxyPort;
+
+        public final InetSocketAddress address;
+        public final Pair<DistributedLogServiceImpl, Server> dlServer;
+        private final SingleHostRoutingService routingService = SingleHostRoutingService.of(null);
+
+        protected DLServer(DistributedLogConfiguration dlConf,
+                           URI uri,
+                           int basePort,
+                           boolean thriftmux) throws Exception {
+            proxyPort = basePort;
+
+            boolean success = false;
+            int retries = 0;
+            Pair<DistributedLogServiceImpl, Server> serverPair = null;
+            while (!success) {
+                try {
+                    org.apache.distributedlog.service.config.ServerConfiguration serverConf =
+                            new org.apache.distributedlog.service.config.ServerConfiguration();
+                    serverConf.loadConf(dlConf);
+                    serverConf.setServerShardId(proxyPort);
+                    serverPair = DistributedLogServer.runServer(
+                            serverConf,
+                            dlConf,
+                            uri,
+                            new IdentityStreamPartitionConverter(),
+                            routingService,
+                            new NullStatsProvider(),
+                            proxyPort,
+                            thriftmux,
+                            new EqualLoadAppraiser());
+                    routingService.setAddress(DLSocketAddress.getSocketAddress(proxyPort));
+                    routingService.startService();
+                    serverPair.getLeft().startPlacementPolicy();
+                    success = true;
+                } catch (BindException be) {
+                    retries++;
+                    if (retries > MAX_RETRIES) {
+                        throw be;
+                    }
+                    proxyPort++;
+                    if (proxyPort > MAX_PORT) {
+                        proxyPort = MIN_PORT;
+                    }
+                }
+            }
+
+            LOG.info("Running DL on port {}", proxyPort);
+
+            dlServer = serverPair;
+            address = DLSocketAddress.getSocketAddress(proxyPort);
+        }
+
+        public InetSocketAddress getAddress() {
+            return address;
+        }
+
+        public void shutdown() {
+            DistributedLogServer.closeServer(dlServer, 0, TimeUnit.MILLISECONDS);
+            routingService.stopService();
+        }
+    }
+
+    private final DistributedLogConfiguration dlConf;
+    private final ZooKeeperServerShim zks;
+    private final LocalDLMEmulator dlmEmulator;
+    private DLServer dlServer;
+    private final boolean shouldStartProxy;
+    private final int proxyPort;
+    private final boolean thriftmux;
+    private final List<File> tmpDirs = new ArrayList<File>();
+
+    private DistributedLogCluster(DistributedLogConfiguration dlConf,
+                                  ServerConfiguration bkConf,
+                                  int numBookies,
+                                  boolean shouldStartZK,
+                                  String zkServers,
+                                  int zkPort,
+                                  boolean shouldStartProxy,
+                                  int proxyPort,
+                                  boolean thriftmux) throws Exception {
+        this.dlConf = dlConf;
+        if (shouldStartZK) {
+            File zkTmpDir = IOUtils.createTempDir("zookeeper", "distrlog");
+            tmpDirs.add(zkTmpDir);
+            if (0 == zkPort) {
+                Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkTmpDir);
+                this.zks = serverAndPort.getLeft();
+                zkPort = serverAndPort.getRight();
+            } else {
+                this.zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir);
+            }
+        } else {
+            this.zks = null;
+        }
+        this.dlmEmulator = LocalDLMEmulator.newBuilder()
+                .numBookies(numBookies)
+                .zkHost(zkServers)
+                .zkPort(zkPort)
+                .serverConf(bkConf)
+                .shouldStartZK(false)
+                .build();
+        this.shouldStartProxy = shouldStartProxy;
+        this.proxyPort = proxyPort;
+        this.thriftmux = thriftmux;
+    }
+
+    public void start() throws Exception {
+        this.dlmEmulator.start();
+        BKDLConfig bkdlConfig = new BKDLConfig(this.dlmEmulator.getZkServers(), "/ledgers").setACLRootPath(".acl");
+        DLMetadata.create(bkdlConfig).update(this.dlmEmulator.getUri());
+        if (shouldStartProxy) {
+            this.dlServer = new DLServer(
+                    dlConf,
+                    this.dlmEmulator.getUri(),
+                    proxyPort,
+                    thriftmux);
+        } else {
+            this.dlServer = null;
+        }
+    }
+
+    public void stop() throws Exception {
+        if (null != dlServer) {
+            this.dlServer.shutdown();
+        }
+        this.dlmEmulator.teardown();
+        if (null != this.zks) {
+            this.zks.stop();
+        }
+        for (File dir : tmpDirs) {
+            FileUtils.forceDeleteOnExit(dir);
+        }
+    }
+
+    public URI getUri() {
+        return this.dlmEmulator.getUri();
+    }
+
+    public String getZkServers() {
+        return this.dlmEmulator.getZkServers();
+    }
+
+    public String getProxyFinagleStr() {
+        return "inet!" + (dlServer == null ? "127.0.0.1:" + proxyPort : dlServer.getAddress().toString());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
new file mode 100644
index 0000000..81e476b
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
@@ -0,0 +1,460 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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 com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.config.DynamicConfigurationFactory;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.service.announcer.Announcer;
+import org.apache.distributedlog.service.announcer.NOPAnnouncer;
+import org.apache.distributedlog.service.announcer.ServerSetAnnouncer;
+import org.apache.distributedlog.service.config.DefaultStreamConfigProvider;
+import org.apache.distributedlog.service.config.NullStreamConfigProvider;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.config.ServiceStreamConfigProvider;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
+import org.apache.distributedlog.service.placement.LoadAppraiser;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import org.apache.distributedlog.util.ConfUtils;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.finagle.Stack;
+import com.twitter.finagle.ThriftMuxServer$;
+import com.twitter.finagle.builder.Server;
+import com.twitter.finagle.builder.ServerBuilder;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientIdRequiredFilter;
+import com.twitter.finagle.thrift.ThriftServerFramedCodec;
+import com.twitter.finagle.transport.Transport;
+import com.twitter.util.Duration;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+
+/**
+ * Running the distributedlog proxy server.
+ */
+public class DistributedLogServer {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServer.class);
+    private static final String DEFAULT_LOAD_APPRIASER = EqualLoadAppraiser.class.getCanonicalName();
+
+    private DistributedLogServiceImpl dlService = null;
+    private Server server = null;
+    private RoutingService routingService;
+    private StatsProvider statsProvider;
+    private Announcer announcer = null;
+    private ScheduledExecutorService configExecutorService;
+    private long gracefulShutdownMs = 0L;
+
+    private final StatsReceiver statsReceiver;
+    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+    private final Optional<String> uri;
+    private final Optional<String> conf;
+    private final Optional<String> streamConf;
+    private final Optional<Integer> port;
+    private final Optional<Integer> statsPort;
+    private final Optional<Integer> shardId;
+    private final Optional<Boolean> announceServerSet;
+    private final Optional<String> loadAppraiserClassStr;
+    private final Optional<Boolean> thriftmux;
+
+    DistributedLogServer(Optional<String> uri,
+                         Optional<String> conf,
+                         Optional<String> streamConf,
+                         Optional<Integer> port,
+                         Optional<Integer> statsPort,
+                         Optional<Integer> shardId,
+                         Optional<Boolean> announceServerSet,
+                         Optional<String> loadAppraiserClass,
+                         Optional<Boolean> thriftmux,
+                         RoutingService routingService,
+                         StatsReceiver statsReceiver,
+                         StatsProvider statsProvider) {
+        this.uri = uri;
+        this.conf = conf;
+        this.streamConf = streamConf;
+        this.port = port;
+        this.statsPort = statsPort;
+        this.shardId = shardId;
+        this.announceServerSet = announceServerSet;
+        this.thriftmux = thriftmux;
+        this.routingService = routingService;
+        this.statsReceiver = statsReceiver;
+        this.statsProvider = statsProvider;
+        this.loadAppraiserClassStr = loadAppraiserClass;
+    }
+
+    public void runServer()
+        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
+        if (!uri.isPresent()) {
+            throw new IllegalArgumentException("No distributedlog uri provided.");
+        }
+        URI dlUri = URI.create(uri.get());
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+        if (conf.isPresent()) {
+            String configFile = conf.get();
+            try {
+                dlConf.loadConf(new File(configFile).toURI().toURL());
+            } catch (ConfigurationException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
+                    + configFile + ".");
+            } catch (MalformedURLException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
+                        + configFile + ".");
+            }
+        }
+
+        this.configExecutorService = Executors.newScheduledThreadPool(1,
+                new ThreadFactoryBuilder()
+                        .setNameFormat("DistributedLogService-Dyncfg-%d")
+                        .setDaemon(true)
+                        .build());
+
+        // server configuration and dynamic configuration
+        ServerConfiguration serverConf = new ServerConfiguration();
+        serverConf.loadConf(dlConf);
+
+        // overwrite the shard id if it is provided in the args
+        if (shardId.isPresent()) {
+            serverConf.setServerShardId(shardId.get());
+        }
+
+        serverConf.validate();
+
+        DynamicDistributedLogConfiguration dynDlConf = getServiceDynConf(dlConf);
+
+        logger.info("Starting stats provider : {}", statsProvider.getClass());
+        statsProvider.start(dlConf);
+
+        if (announceServerSet.isPresent() && announceServerSet.get()) {
+            announcer = new ServerSetAnnouncer(
+                    dlUri,
+                    port.or(0),
+                    statsPort.or(0),
+                    shardId.or(0));
+        } else {
+            announcer = new NOPAnnouncer();
+        }
+
+        // Build the stream partition converter
+        StreamPartitionConverter converter;
+        try {
+            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
+        } catch (ConfigurationException e) {
+            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
+                    IdentityStreamPartitionConverter.class.getName());
+            converter = new IdentityStreamPartitionConverter();
+        }
+        Class loadAppraiserClass = Class.forName(loadAppraiserClassStr.or(DEFAULT_LOAD_APPRIASER));
+        LoadAppraiser loadAppraiser = (LoadAppraiser) ReflectionUtils.newInstance(loadAppraiserClass);
+        logger.info("Load appraiser class is " + loadAppraiserClassStr.or("not specified.") + " Instantiated "
+                + loadAppraiser.getClass().getCanonicalName());
+
+        StreamConfigProvider streamConfProvider =
+                getStreamConfigProvider(dlConf, converter);
+
+        // pre-run
+        preRun(dlConf, serverConf);
+
+        Pair<DistributedLogServiceImpl, Server> serverPair = runServer(
+                serverConf,
+                dlConf,
+                dynDlConf,
+                dlUri,
+                converter,
+                routingService,
+                statsProvider,
+                port.or(0),
+                keepAliveLatch,
+                statsReceiver,
+                thriftmux.isPresent(),
+                streamConfProvider,
+                loadAppraiser);
+
+        this.dlService = serverPair.getLeft();
+        this.server = serverPair.getRight();
+
+        // announce the service
+        announcer.announce();
+        // start the routing service after announced
+        routingService.startService();
+        logger.info("Started the routing service.");
+        dlService.startPlacementPolicy();
+        logger.info("Started the placement policy.");
+    }
+
+    protected void preRun(DistributedLogConfiguration conf, ServerConfiguration serverConf) {
+        this.gracefulShutdownMs = serverConf.getGracefulShutdownPeriodMs();
+        if (!serverConf.isDurableWriteEnabled()) {
+            conf.setDurableWriteEnabled(false);
+        }
+    }
+
+    private DynamicDistributedLogConfiguration getServiceDynConf(DistributedLogConfiguration dlConf)
+        throws ConfigurationException {
+        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.absent();
+        if (conf.isPresent()) {
+            DynamicConfigurationFactory configFactory = new DynamicConfigurationFactory(
+                    configExecutorService, dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
+            dynConf = configFactory.getDynamicConfiguration(conf.get());
+        }
+        if (dynConf.isPresent()) {
+            return dynConf.get();
+        } else {
+            return ConfUtils.getConstDynConf(dlConf);
+        }
+    }
+
+    private StreamConfigProvider getStreamConfigProvider(DistributedLogConfiguration dlConf,
+                                                         StreamPartitionConverter partitionConverter)
+            throws ConfigurationException {
+        StreamConfigProvider streamConfProvider = new NullStreamConfigProvider();
+        if (streamConf.isPresent() && conf.isPresent()) {
+            String dynConfigPath = streamConf.get();
+            String defaultConfigFile = conf.get();
+            streamConfProvider = new ServiceStreamConfigProvider(
+                    dynConfigPath,
+                    defaultConfigFile,
+                    partitionConverter,
+                    configExecutorService,
+                    dlConf.getDynamicConfigReloadIntervalSec(),
+                    TimeUnit.SECONDS);
+        } else if (conf.isPresent()) {
+            String configFile = conf.get();
+            streamConfProvider = new DefaultStreamConfigProvider(configFile, configExecutorService,
+                    dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
+        }
+        return streamConfProvider;
+    }
+
+    static Pair<DistributedLogServiceImpl, Server> runServer(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf,
+            URI dlUri,
+            StreamPartitionConverter converter,
+            RoutingService routingService,
+            StatsProvider provider,
+            int port,
+            boolean thriftmux,
+            LoadAppraiser loadAppraiser) throws IOException {
+
+        return runServer(serverConf,
+                dlConf,
+                ConfUtils.getConstDynConf(dlConf),
+                dlUri,
+                converter,
+                routingService,
+                provider,
+                port,
+                new CountDownLatch(0),
+                new NullStatsReceiver(),
+                thriftmux,
+                new NullStreamConfigProvider(),
+                loadAppraiser);
+    }
+
+    static Pair<DistributedLogServiceImpl, Server> runServer(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf,
+            DynamicDistributedLogConfiguration dynDlConf,
+            URI dlUri,
+            StreamPartitionConverter partitionConverter,
+            RoutingService routingService,
+            StatsProvider provider,
+            int port,
+            CountDownLatch keepAliveLatch,
+            StatsReceiver statsReceiver,
+            boolean thriftmux,
+            StreamConfigProvider streamConfProvider,
+            LoadAppraiser loadAppraiser) throws IOException {
+        logger.info("Running server @ uri {}.", dlUri);
+
+        boolean perStreamStatsEnabled = serverConf.isPerStreamStatEnabled();
+        StatsLogger perStreamStatsLogger;
+        if (perStreamStatsEnabled) {
+            perStreamStatsLogger = provider.getStatsLogger("stream");
+        } else {
+            perStreamStatsLogger = NullStatsLogger.INSTANCE;
+        }
+
+        // dl service
+        DistributedLogServiceImpl dlService = new DistributedLogServiceImpl(
+            serverConf,
+            dlConf,
+            dynDlConf,
+            streamConfProvider,
+            dlUri,
+            partitionConverter,
+            routingService,
+            provider.getStatsLogger(""),
+            perStreamStatsLogger,
+            keepAliveLatch,
+            loadAppraiser);
+
+        StatsReceiver serviceStatsReceiver = statsReceiver.scope("service");
+        StatsLogger serviceStatsLogger = provider.getStatsLogger("service");
+
+        ServerBuilder serverBuilder = ServerBuilder.get()
+                .name("DistributedLogServer")
+                .codec(ThriftServerFramedCodec.get())
+                .reportTo(statsReceiver)
+                .keepAlive(true)
+                .bindTo(new InetSocketAddress(port));
+
+        if (thriftmux) {
+            logger.info("Using thriftmux.");
+            Tuple2<Transport.Liveness, Stack.Param<Transport.Liveness>> livenessParam = new Transport.Liveness(
+                    Duration.Top(), Duration.Top(), Option.apply((Object) Boolean.valueOf(true))).mk();
+            serverBuilder = serverBuilder.stack(
+                ThriftMuxServer$.MODULE$.configured(livenessParam._1(), livenessParam._2()));
+        }
+
+        logger.info("DistributedLogServer running with the following configuration : \n{}", dlConf.getPropsAsString());
+
+        // starts dl server
+        Server server = ServerBuilder.safeBuild(
+                new ClientIdRequiredFilter<byte[], byte[]>(serviceStatsReceiver).andThen(
+                    new StatsFilter<byte[], byte[]>(serviceStatsLogger).andThen(
+                        new DistributedLogService.Service(dlService, new TBinaryProtocol.Factory()))),
+                serverBuilder);
+
+        logger.info("Started DistributedLog Server.");
+        return Pair.of(dlService, server);
+    }
+
+    static void closeServer(Pair<DistributedLogServiceImpl, Server> pair,
+                            long gracefulShutdownPeriod,
+                            TimeUnit timeUnit) {
+        if (null != pair.getLeft()) {
+            pair.getLeft().shutdown();
+            if (gracefulShutdownPeriod > 0) {
+                try {
+                    timeUnit.sleep(gracefulShutdownPeriod);
+                } catch (InterruptedException e) {
+                    logger.info("Interrupted on waiting service shutting down state propagated to all clients : ", e);
+                }
+            }
+        }
+        if (null != pair.getRight()) {
+            logger.info("Closing dl thrift server.");
+            pair.getRight().close();
+            logger.info("Closed dl thrift server.");
+        }
+    }
+
+    /**
+     * Close the server.
+     */
+    public void close() {
+        if (null != announcer) {
+            try {
+                announcer.unannounce();
+            } catch (IOException e) {
+                logger.warn("Error on unannouncing service : ", e);
+            }
+            announcer.close();
+        }
+        closeServer(Pair.of(dlService, server), gracefulShutdownMs, TimeUnit.MILLISECONDS);
+        routingService.stopService();
+        if (null != statsProvider) {
+            statsProvider.stop();
+        }
+        SchedulerUtils.shutdownScheduler(configExecutorService, 60, TimeUnit.SECONDS);
+        keepAliveLatch.countDown();
+    }
+
+    public void join() throws InterruptedException {
+        keepAliveLatch.await();
+    }
+
+    /**
+     * Running distributedlog server.
+     *
+     * @param uri distributedlog namespace
+     * @param conf distributedlog configuration file location
+     * @param streamConf per stream configuration dir location
+     * @param port listen port
+     * @param statsPort stats port
+     * @param shardId shard id
+     * @param announceServerSet whether to announce itself to server set
+     * @param thriftmux flag to enable thrift mux
+     * @param statsReceiver receiver to receive finagle stats
+     * @param statsProvider provider to receive dl stats
+     * @return distributedlog server
+     * @throws ConfigurationException
+     * @throws IllegalArgumentException
+     * @throws IOException
+     * @throws ClassNotFoundException
+     */
+    public static DistributedLogServer runServer(
+               Optional<String> uri,
+               Optional<String> conf,
+               Optional<String> streamConf,
+               Optional<Integer> port,
+               Optional<Integer> statsPort,
+               Optional<Integer> shardId,
+               Optional<Boolean> announceServerSet,
+               Optional<String> loadAppraiserClass,
+               Optional<Boolean> thriftmux,
+               RoutingService routingService,
+               StatsReceiver statsReceiver,
+               StatsProvider statsProvider)
+        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
+
+        final DistributedLogServer server = new DistributedLogServer(
+                uri,
+                conf,
+                streamConf,
+                port,
+                statsPort,
+                shardId,
+                announceServerSet,
+                loadAppraiserClass,
+                thriftmux,
+                routingService,
+                statsReceiver,
+                statsProvider);
+
+        server.runServer();
+        return server;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
new file mode 100644
index 0000000..a1642f9
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.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.service;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalStringArg;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.client.routing.RoutingUtils;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+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.cli.ParseException;
+import org.apache.commons.configuration.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The launcher of the distributedlog proxy server.
+ */
+public class DistributedLogServerApp {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServerApp.class);
+
+    private static final String USAGE = "DistributedLogServerApp [-u <uri>] [-c <conf>]";
+    private final String[] args;
+    private final Options options = new Options();
+
+    private DistributedLogServerApp(String[] args) {
+        this.args = args;
+
+        // prepare options
+        options.addOption("u", "uri", true, "DistributedLog URI");
+        options.addOption("c", "conf", true, "DistributedLog Configuration File");
+        options.addOption("sc", "stream-conf", true, "Per Stream Configuration Directory");
+        options.addOption("p", "port", true, "DistributedLog Server Port");
+        options.addOption("sp", "stats-port", true, "DistributedLog Stats Port");
+        options.addOption("pd", "stats-provider", true, "DistributedLog Stats Provider");
+        options.addOption("si", "shard-id", true, "DistributedLog Shard ID");
+        options.addOption("a", "announce", false, "ServerSet Path to Announce");
+        options.addOption("la", "load-appraiser", true, "LoadAppraiser Implementation to Use");
+        options.addOption("mx", "thriftmux", false, "Is thriftmux enabled");
+    }
+
+    private void printUsage() {
+        HelpFormatter helpFormatter = new HelpFormatter();
+        helpFormatter.printHelp(USAGE, options);
+    }
+
+    private void run() {
+        try {
+            logger.info("Running distributedlog server : args = {}", Arrays.toString(args));
+            BasicParser parser = new BasicParser();
+            CommandLine cmdline = parser.parse(options, args);
+            runCmd(cmdline);
+        } catch (ParseException pe) {
+            logger.error("Argument error : {}", pe.getMessage());
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (IllegalArgumentException iae) {
+            logger.error("Argument error : {}", iae.getMessage());
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (ConfigurationException ce) {
+            logger.error("Configuration error : {}", ce.getMessage());
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (IOException ie) {
+            logger.error("Failed to start distributedlog server : ", ie);
+            Runtime.getRuntime().exit(-1);
+        } catch (ClassNotFoundException cnf) {
+          logger.error("Failed to start distributedlog server : ", cnf);
+          Runtime.getRuntime().exit(-1);
+        }
+    }
+
+    private void runCmd(CommandLine cmdline)
+        throws IllegalArgumentException, IOException, ConfigurationException, ClassNotFoundException {
+        final StatsReceiver statsReceiver = NullStatsReceiver.get();
+        Optional<String> confOptional = getOptionalStringArg(cmdline, "c");
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+        if (confOptional.isPresent()) {
+            String configFile = confOptional.get();
+            try {
+                dlConf.loadConf(new File(configFile).toURI().toURL());
+            } catch (ConfigurationException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
+                    + configFile + ".");
+            } catch (MalformedURLException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
+                        + configFile + ".");
+            }
+        }
+        // load the stats provider
+        final StatsProvider statsProvider = getOptionalStringArg(cmdline, "pd")
+                .transform(new Function<String, StatsProvider>() {
+                    @Nullable
+                    @Override
+                    public StatsProvider apply(@Nullable String name) {
+                        return ReflectionUtils.newInstance(name, StatsProvider.class);
+                    }
+                }).or(new NullStatsProvider());
+
+        final Optional<String> uriOption = getOptionalStringArg(cmdline, "u");
+        checkArgument(uriOption.isPresent(), "No distributedlog uri provided.");
+        URI dlUri = URI.create(uriOption.get());
+
+        DLZkServerSet serverSet = DLZkServerSet.of(dlUri, (int) TimeUnit.SECONDS.toMillis(60));
+        RoutingService routingService = RoutingUtils.buildRoutingService(serverSet.getServerSet())
+                .statsReceiver(statsReceiver.scope("routing"))
+                .build();
+
+        final DistributedLogServer server = DistributedLogServer.runServer(
+                uriOption,
+                confOptional,
+                getOptionalStringArg(cmdline, "sc"),
+                getOptionalIntegerArg(cmdline, "p"),
+                getOptionalIntegerArg(cmdline, "sp"),
+                getOptionalIntegerArg(cmdline, "si"),
+                getOptionalBooleanArg(cmdline, "a"),
+                getOptionalStringArg(cmdline, "la"),
+                getOptionalBooleanArg(cmdline, "mx"),
+                routingService,
+                statsReceiver,
+                statsProvider);
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                logger.info("Closing DistributedLog Server.");
+                server.close();
+                logger.info("Closed DistributedLog Server.");
+                statsProvider.stop();
+            }
+        });
+
+        try {
+            server.join();
+        } catch (InterruptedException e) {
+            logger.warn("Interrupted when waiting distributedlog server to be finished : ", e);
+        }
+
+        logger.info("DistributedLog Service Interrupted.");
+        server.close();
+        logger.info("Closed DistributedLog Server.");
+        statsProvider.stop();
+    }
+
+    public static void main(String[] args) {
+        final DistributedLogServerApp launcher = new DistributedLogServerApp(args);
+        launcher.run();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
new file mode 100644
index 0000000..c37cd53
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
@@ -0,0 +1,794 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.twitter.common.net.InetSocketAddressHelper;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.RegionUnavailableException;
+import org.apache.distributedlog.exceptions.ServiceUnavailableException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.exceptions.TooManyStreamsException;
+import org.apache.distributedlog.feature.AbstractFeatureProvider;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.rate.MovingAverageRate;
+import org.apache.distributedlog.rate.MovingAverageRateFactory;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.placement.LeastLoadPlacementPolicy;
+import org.apache.distributedlog.service.placement.LoadAppraiser;
+import org.apache.distributedlog.service.placement.PlacementPolicy;
+import org.apache.distributedlog.service.placement.ZKPlacementStateManager;
+import org.apache.distributedlog.service.stream.BulkWriteOp;
+import org.apache.distributedlog.service.stream.DeleteOp;
+import org.apache.distributedlog.service.stream.HeartbeatOp;
+import org.apache.distributedlog.service.stream.ReleaseOp;
+import org.apache.distributedlog.service.stream.Stream;
+import org.apache.distributedlog.service.stream.StreamFactory;
+import org.apache.distributedlog.service.stream.StreamFactoryImpl;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamManagerImpl;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.distributedlog.service.stream.StreamOpStats;
+import org.apache.distributedlog.service.stream.TruncateOp;
+import org.apache.distributedlog.service.stream.WriteOp;
+import org.apache.distributedlog.service.stream.WriteOpWithPayload;
+import org.apache.distributedlog.service.stream.admin.CreateOp;
+import org.apache.distributedlog.service.stream.admin.StreamAdminOp;
+import org.apache.distributedlog.service.stream.limiter.ServiceRequestLimiter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.service.utils.ServerUtils;
+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.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.ConfUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.util.Await;
+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.ScheduledThreadPoolTimer;
+import com.twitter.util.Timer;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Implementation of distributedlog thrift service.
+ */
+public class DistributedLogServiceImpl implements DistributedLogService.ServiceIface,
+                                                  FatalErrorHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServiceImpl.class);
+
+    private static final int MOVING_AVERAGE_WINDOW_SECS = 60;
+
+    private final ServerConfiguration serverConfig;
+    private final DistributedLogConfiguration dlConfig;
+    private final DistributedLogNamespace dlNamespace;
+    private final int serverRegionId;
+    private final PlacementPolicy placementPolicy;
+    private ServerStatus serverStatus = ServerStatus.WRITE_AND_ACCEPT;
+    private final ReentrantReadWriteLock closeLock =
+            new ReentrantReadWriteLock();
+    private final CountDownLatch keepAliveLatch;
+    private final byte dlsnVersion;
+    private final String clientId;
+    private final OrderedScheduler scheduler;
+    private final AccessControlManager accessControlManager;
+    private final StreamConfigProvider streamConfigProvider;
+    private final StreamManager streamManager;
+    private final StreamFactory streamFactory;
+    private final RoutingService routingService;
+    private final RegionResolver regionResolver;
+    private final MovingAverageRateFactory movingAvgFactory;
+    private final MovingAverageRate windowedRps;
+    private final MovingAverageRate windowedBps;
+    private final ServiceRequestLimiter limiter;
+    private final Timer timer;
+    private final HashedWheelTimer requestTimer;
+
+    // Features
+    private final FeatureProvider featureProvider;
+    private final Feature featureRegionStopAcceptNewStream;
+    private final Feature featureChecksumDisabled;
+    private final Feature limiterDisabledFeature;
+
+    // Stats
+    private final StatsLogger statsLogger;
+    private final StatsLogger perStreamStatsLogger;
+    private final StreamPartitionConverter streamPartitionConverter;
+    private final StreamOpStats streamOpStats;
+    private final Counter bulkWritePendingStat;
+    private final Counter writePendingStat;
+    private final Counter redirects;
+    private final Counter receivedRecordCounter;
+    private final StatsLogger statusCodeStatLogger;
+    private final ConcurrentHashMap<StatusCode, Counter> statusCodeCounters =
+            new ConcurrentHashMap<StatusCode, Counter>();
+    private final Counter statusCodeTotal;
+    private final Gauge<Number> proxyStatusGauge;
+    private final Gauge<Number> movingAvgRpsGauge;
+    private final Gauge<Number> movingAvgBpsGauge;
+    private final Gauge<Number> streamAcquiredGauge;
+    private final Gauge<Number> streamCachedGauge;
+    private final int shard;
+
+    DistributedLogServiceImpl(ServerConfiguration serverConf,
+                              DistributedLogConfiguration dlConf,
+                              DynamicDistributedLogConfiguration dynDlConf,
+                              StreamConfigProvider streamConfigProvider,
+                              URI uri,
+                              StreamPartitionConverter converter,
+                              RoutingService routingService,
+                              StatsLogger statsLogger,
+                              StatsLogger perStreamStatsLogger,
+                              CountDownLatch keepAliveLatch,
+                              LoadAppraiser loadAppraiser)
+            throws IOException {
+        // Configuration.
+        this.serverConfig = serverConf;
+        this.dlConfig = dlConf;
+        this.perStreamStatsLogger = perStreamStatsLogger;
+        this.dlsnVersion = serverConf.getDlsnVersion();
+        this.serverRegionId = serverConf.getRegionId();
+        this.streamPartitionConverter = converter;
+        int serverPort = serverConf.getServerPort();
+        this.shard = serverConf.getServerShardId();
+        int numThreads = serverConf.getServerThreads();
+        this.clientId = DLSocketAddress.toLockId(DLSocketAddress.getSocketAddress(serverPort), shard);
+        String allocatorPoolName = ServerUtils.getLedgerAllocatorPoolName(
+            serverRegionId,
+            shard,
+            serverConf.isUseHostnameAsAllocatorPoolName());
+        dlConf.setLedgerAllocatorPoolName(allocatorPoolName);
+        this.featureProvider = AbstractFeatureProvider.getFeatureProvider("", dlConf, statsLogger.scope("features"));
+        if (this.featureProvider instanceof AbstractFeatureProvider) {
+            ((AbstractFeatureProvider) featureProvider).start();
+        }
+
+        // Build the namespace
+        this.dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(dlConf)
+                .uri(uri)
+                .statsLogger(statsLogger)
+                .featureProvider(this.featureProvider)
+                .clientId(clientId)
+                .regionId(serverRegionId)
+                .build();
+        this.accessControlManager = this.dlNamespace.createAccessControlManager();
+        this.keepAliveLatch = keepAliveLatch;
+        this.streamConfigProvider = streamConfigProvider;
+
+        // Stats pertaining to stream op execution
+        this.streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+
+        // Executor Service.
+        this.scheduler = OrderedScheduler.newBuilder()
+                .corePoolSize(numThreads)
+                .name("DistributedLogService-Executor")
+                .traceTaskExecution(true)
+                .statsLogger(statsLogger.scope("scheduler"))
+                .build();
+
+        // Timer, kept separate to ensure reliability of timeouts.
+        this.requestTimer = new HashedWheelTimer(
+            new ThreadFactoryBuilder().setNameFormat("DLServiceTimer-%d").build(),
+            dlConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
+            dlConf.getTimeoutTimerNumTicks());
+
+        // Creating and managing Streams
+        this.streamFactory = new StreamFactoryImpl(clientId,
+                streamOpStats,
+                serverConf,
+                dlConf,
+                featureProvider,
+                streamConfigProvider,
+                converter,
+                dlNamespace,
+                scheduler,
+                this,
+                requestTimer);
+        this.streamManager = new StreamManagerImpl(
+                clientId,
+                dlConf,
+                scheduler,
+                streamFactory,
+                converter,
+                streamConfigProvider,
+                dlNamespace);
+        this.routingService = routingService;
+        this.regionResolver = new DefaultRegionResolver();
+
+        // Service features
+        this.featureRegionStopAcceptNewStream = this.featureProvider.getFeature(
+                ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase());
+        this.featureChecksumDisabled = this.featureProvider.getFeature(
+                ServerFeatureKeys.SERVICE_CHECKSUM_DISABLED.name().toLowerCase());
+        this.limiterDisabledFeature = this.featureProvider.getFeature(
+                ServerFeatureKeys.SERVICE_GLOBAL_LIMITER_DISABLED.name().toLowerCase());
+
+        // Resource limiting
+        this.timer = new ScheduledThreadPoolTimer(1, "timer", true);
+        this.movingAvgFactory = new MovingAverageRateFactory(timer);
+        this.windowedRps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
+        this.windowedBps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
+        this.limiter = new ServiceRequestLimiter(
+                dynDlConf,
+                streamOpStats.baseScope("service_limiter"),
+                windowedRps,
+                windowedBps,
+                streamManager,
+                limiterDisabledFeature);
+
+        this.placementPolicy = new LeastLoadPlacementPolicy(
+            loadAppraiser,
+            routingService,
+            dlNamespace,
+            new ZKPlacementStateManager(uri, dlConf, statsLogger),
+            Duration.fromSeconds(serverConf.getResourcePlacementRefreshInterval()),
+            statsLogger);
+        logger.info("placement started");
+
+        // Stats
+        this.statsLogger = statsLogger;
+
+        // Gauges for server status/health
+        this.proxyStatusGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return ServerStatus.DOWN == serverStatus ? -1 : (featureRegionStopAcceptNewStream.isAvailable()
+                    ? 3 : (ServerStatus.WRITE_AND_ACCEPT == serverStatus ? 1 : 2));
+            }
+        };
+        this.movingAvgRpsGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return windowedRps.get();
+            }
+        };
+        this.movingAvgBpsGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return windowedBps.get();
+            }
+        };
+        // Gauges for streams
+        this.streamAcquiredGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return streamManager.numAcquired();
+            }
+        };
+        this.streamCachedGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return streamManager.numCached();
+            }
+        };
+
+        // Stats on server
+        statsLogger.registerGauge("proxy_status", proxyStatusGauge);
+        // Global moving average rps
+        statsLogger.registerGauge("moving_avg_rps", movingAvgRpsGauge);
+        // Global moving average bps
+        statsLogger.registerGauge("moving_avg_bps", movingAvgBpsGauge);
+        // Stats on requests
+        this.bulkWritePendingStat = streamOpStats.requestPendingCounter("bulkWritePending");
+        this.writePendingStat = streamOpStats.requestPendingCounter("writePending");
+        this.redirects = streamOpStats.requestCounter("redirect");
+        this.statusCodeStatLogger = streamOpStats.requestScope("statuscode");
+        this.statusCodeTotal = streamOpStats.requestCounter("statuscode_count");
+        this.receivedRecordCounter = streamOpStats.recordsCounter("received");
+
+        // Stats for streams
+        StatsLogger streamsStatsLogger = statsLogger.scope("streams");
+        streamsStatsLogger.registerGauge("acquired", this.streamAcquiredGauge);
+        streamsStatsLogger.registerGauge("cached", this.streamCachedGauge);
+
+        // Setup complete
+        logger.info("Running distributedlog server : client id {}, allocator pool {}, perstream stat {},"
+            + " dlsn version {}.",
+            new Object[] { clientId, allocatorPoolName, serverConf.isPerStreamStatEnabled(), dlsnVersion });
+    }
+
+    private void countStatusCode(StatusCode code) {
+        Counter counter = statusCodeCounters.get(code);
+        if (null == counter) {
+            counter = statusCodeStatLogger.getCounter(code.name());
+            Counter oldCounter = statusCodeCounters.putIfAbsent(code, counter);
+            if (null != oldCounter) {
+                counter = oldCounter;
+            }
+        }
+        counter.inc();
+        statusCodeTotal.inc();
+    }
+
+    @Override
+    public Future<ServerInfo> handshake() {
+        return handshakeWithClientInfo(new ClientInfo());
+    }
+
+    @Override
+    public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
+        ServerInfo serverInfo = new ServerInfo();
+        closeLock.readLock().lock();
+        try {
+            serverInfo.setServerStatus(serverStatus);
+        } finally {
+            closeLock.readLock().unlock();
+        }
+
+        if (clientInfo.isSetGetOwnerships() && !clientInfo.isGetOwnerships()) {
+            return Future.value(serverInfo);
+        }
+
+        Optional<String> regex = Optional.absent();
+        if (clientInfo.isSetStreamNameRegex()) {
+            regex = Optional.of(clientInfo.getStreamNameRegex());
+        }
+
+        Map<String, String> ownershipMap = streamManager.getStreamOwnershipMap(regex);
+        serverInfo.setOwnerships(ownershipMap);
+        return Future.value(serverInfo);
+    }
+
+    @VisibleForTesting
+    Stream getLogWriter(String stream) throws IOException {
+        Stream writer = streamManager.getStream(stream);
+        if (null == writer) {
+            closeLock.readLock().lock();
+            try {
+                if (featureRegionStopAcceptNewStream.isAvailable()) {
+                    // accept new stream is disabled in current dc
+                    throw new RegionUnavailableException("Region is unavailable right now.");
+                } else if (!(ServerStatus.WRITE_AND_ACCEPT == serverStatus)) {
+                    // if it is closed, we would not acquire stream again.
+                    return null;
+                }
+                writer = streamManager.getOrCreateStream(stream, true);
+            } finally {
+                closeLock.readLock().unlock();
+            }
+        }
+        return writer;
+    }
+
+    // Service interface methods
+
+    @Override
+    public Future<WriteResponse> write(final String stream, ByteBuffer data) {
+        receivedRecordCounter.inc();
+        return doWrite(stream, data, null /* checksum */, false);
+    }
+
+    @Override
+    public Future<BulkWriteResponse> writeBulkWithContext(final String stream,
+                                                          List<ByteBuffer> data,
+                                                          WriteContext ctx) {
+        bulkWritePendingStat.inc();
+        receivedRecordCounter.add(data.size());
+        BulkWriteOp op = new BulkWriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
+            getChecksum(ctx), featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result().ensure(new Function0<BoxedUnit>() {
+            public BoxedUnit apply() {
+                bulkWritePendingStat.dec();
+                return null;
+            }
+        });
+    }
+
+    @Override
+    public Future<WriteResponse> writeWithContext(final String stream, ByteBuffer data, WriteContext ctx) {
+        return doWrite(stream, data, getChecksum(ctx), ctx.isIsRecordSet());
+    }
+
+    @Override
+    public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
+        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> heartbeatWithOptions(String stream, WriteContext ctx, HeartbeatOptions options) {
+        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        if (options.isSendHeartBeatToReader()) {
+            op.setWriteControlRecord(true);
+        }
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> truncate(String stream, String dlsn, WriteContext ctx) {
+        TruncateOp op = new TruncateOp(
+            stream,
+            DLSN.deserialize(dlsn),
+            statsLogger,
+            perStreamStatsLogger,
+            getChecksum(ctx),
+            featureChecksumDisabled,
+            accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> delete(String stream, WriteContext ctx) {
+        DeleteOp op = new DeleteOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> release(String stream, WriteContext ctx) {
+        ReleaseOp op = new ReleaseOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> create(String stream, WriteContext ctx) {
+        CreateOp op = new CreateOp(stream, statsLogger, streamManager, getChecksum(ctx), featureChecksumDisabled);
+        return executeStreamAdminOp(op);
+    }
+
+    //
+    // Ownership RPC
+    //
+
+    @Override
+    public Future<WriteResponse> getOwner(String streamName, WriteContext ctx) {
+        if (streamManager.isAcquired(streamName)) {
+            // the stream is already acquired
+            return Future.value(new WriteResponse(ResponseUtils.ownerToHeader(clientId)));
+        }
+
+        return placementPolicy.placeStream(streamName).map(new Function<String, WriteResponse>() {
+            @Override
+            public WriteResponse apply(String server) {
+                String host = DLSocketAddress.toLockId(InetSocketAddressHelper.parse(server), -1);
+                return new WriteResponse(ResponseUtils.ownerToHeader(host));
+            }
+        });
+    }
+
+
+    //
+    // Admin RPCs
+    //
+
+    @Override
+    public Future<Void> setAcceptNewStream(boolean enabled) {
+        closeLock.writeLock().lock();
+        try {
+            logger.info("Set AcceptNewStream = {}", enabled);
+            if (ServerStatus.DOWN != serverStatus) {
+                if (enabled) {
+                    serverStatus = ServerStatus.WRITE_AND_ACCEPT;
+                } else {
+                    serverStatus = ServerStatus.WRITE_ONLY;
+                }
+            }
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+        return Future.Void();
+    }
+
+    private Future<WriteResponse> doWrite(final String name,
+                                          ByteBuffer data,
+                                          Long checksum,
+                                          boolean isRecordSet) {
+        writePendingStat.inc();
+        receivedRecordCounter.inc();
+        WriteOp op = newWriteOp(name, data, checksum, isRecordSet);
+        executeStreamOp(op);
+        return op.result().ensure(new Function0<BoxedUnit>() {
+            public BoxedUnit apply() {
+                writePendingStat.dec();
+                return null;
+            }
+        });
+    }
+
+    private Long getChecksum(WriteContext ctx) {
+        return ctx.isSetCrc32() ? ctx.getCrc32() : null;
+    }
+
+    private Future<WriteResponse> executeStreamAdminOp(final StreamAdminOp op) {
+        try {
+            op.preExecute();
+        } catch (DLException dle) {
+            return Future.exception(dle);
+        }
+        return op.execute();
+    }
+
+    private void executeStreamOp(final StreamOp op) {
+
+        // Must attach this as early as possible--returning before this point will cause us to
+        // lose the status code.
+        op.responseHeader().addEventListener(new FutureEventListener<ResponseHeader>() {
+            @Override
+            public void onSuccess(ResponseHeader header) {
+                if (header.getLocation() != null || header.getCode() == StatusCode.FOUND) {
+                    redirects.inc();
+                }
+                countStatusCode(header.getCode());
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+            }
+        });
+
+        try {
+            // Apply the request limiter
+            limiter.apply(op);
+
+            // Execute per-op pre-exec code
+            op.preExecute();
+
+        } catch (TooManyStreamsException e) {
+            // Translate to StreamUnavailableException to ensure that the client will redirect
+            // to a different host. Ideally we would be able to return TooManyStreamsException,
+            // but the way exception handling works right now we can't control the handling in
+            // the client because client changes deploy very slowly.
+            op.fail(new StreamUnavailableException(e.getMessage()));
+            return;
+        } catch (Exception e) {
+            op.fail(e);
+            return;
+        }
+
+        Stream stream;
+        try {
+            stream = getLogWriter(op.streamName());
+        } catch (RegionUnavailableException rue) {
+            // redirect the requests to other region
+            op.fail(new RegionUnavailableException("Region " + serverRegionId + " is unavailable."));
+            return;
+        } catch (IOException e) {
+            op.fail(e);
+            return;
+        }
+        if (null == stream) {
+            // redirect the requests when stream is unavailable.
+            op.fail(new ServiceUnavailableException("Server " + clientId + " is closed."));
+            return;
+        }
+
+        if (op instanceof WriteOpWithPayload) {
+            WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
+            windowedBps.add(writeOp.getPayloadSize());
+            windowedRps.inc();
+        }
+
+        stream.submit(op);
+    }
+
+    void shutdown() {
+        try {
+            closeLock.writeLock().lock();
+            try {
+                if (ServerStatus.DOWN == serverStatus) {
+                    return;
+                }
+                serverStatus = ServerStatus.DOWN;
+            } finally {
+                closeLock.writeLock().unlock();
+            }
+
+            streamManager.close();
+            movingAvgFactory.close();
+            limiter.close();
+
+            Stopwatch closeStreamsStopwatch = Stopwatch.createStarted();
+
+            Future<List<Void>> closeResult = streamManager.closeStreams();
+            logger.info("Waiting for closing all streams ...");
+            try {
+                Await.result(closeResult, Duration.fromTimeUnit(5, TimeUnit.MINUTES));
+                logger.info("Closed all streams in {} millis.",
+                        closeStreamsStopwatch.elapsed(TimeUnit.MILLISECONDS));
+            } catch (InterruptedException e) {
+                logger.warn("Interrupted on waiting for closing all streams : ", e);
+                Thread.currentThread().interrupt();
+            } catch (Exception e) {
+                logger.warn("Sorry, we didn't close all streams gracefully in 5 minutes : ", e);
+            }
+
+            // shutdown the dl namespace
+            logger.info("Closing distributedlog namespace ...");
+            dlNamespace.close();
+            logger.info("Closed distributedlog namespace .");
+
+            // Stop the feature provider
+            if (this.featureProvider instanceof AbstractFeatureProvider) {
+                ((AbstractFeatureProvider) featureProvider).stop();
+            }
+
+            // Stop the timer.
+            timer.stop();
+            placementPolicy.close();
+
+            // clean up gauge
+            unregisterGauge();
+
+            // shutdown the executor after requesting closing streams.
+            SchedulerUtils.shutdownScheduler(scheduler, 60, TimeUnit.SECONDS);
+        } catch (Exception ex) {
+            logger.info("Exception while shutting down distributedlog service.");
+        } finally {
+            // release the keepAliveLatch in case shutdown is called from a shutdown hook.
+            keepAliveLatch.countDown();
+            logger.info("Finished shutting down distributedlog service.");
+        }
+    }
+
+    protected void startPlacementPolicy() {
+        this.placementPolicy.start(shard == 0);
+    }
+
+    @Override
+    public void notifyFatalError() {
+        triggerShutdown();
+    }
+
+    private void triggerShutdown() {
+        // release the keepAliveLatch to let the main thread shutdown the whole service.
+        logger.info("Releasing KeepAlive Latch to trigger shutdown ...");
+        keepAliveLatch.countDown();
+        logger.info("Released KeepAlive Latch. Main thread will shut the service down.");
+    }
+
+    // Test methods.
+
+    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
+        Optional<DynamicDistributedLogConfiguration> dynDlConf =
+                streamConfigProvider.getDynamicStreamConfig(streamName);
+        if (dynDlConf.isPresent()) {
+            return dynDlConf.get();
+        } else {
+            return ConfUtils.getConstDynConf(dlConfig);
+        }
+    }
+
+    /**
+     * clean up the gauge before we close to help GC.
+     */
+    private void unregisterGauge(){
+        this.statsLogger.unregisterGauge("proxy_status", this.proxyStatusGauge);
+        this.statsLogger.unregisterGauge("moving_avg_rps", this.movingAvgRpsGauge);
+        this.statsLogger.unregisterGauge("moving_avg_bps", this.movingAvgBpsGauge);
+        this.statsLogger.unregisterGauge("acquired", this.streamAcquiredGauge);
+        this.statsLogger.unregisterGauge("cached", this.streamCachedGauge);
+    }
+
+    @VisibleForTesting
+    Stream newStream(String name) throws IOException {
+        return streamManager.getOrCreateStream(name, false);
+    }
+
+    @VisibleForTesting
+    WriteOp newWriteOp(String stream, ByteBuffer data, Long checksum) {
+        return newWriteOp(stream, data, checksum, false);
+    }
+
+    @VisibleForTesting
+    RoutingService getRoutingService() {
+        return this.routingService;
+    }
+
+    @VisibleForTesting
+    DLSocketAddress getServiceAddress() throws IOException {
+        return DLSocketAddress.deserialize(clientId);
+    }
+
+    WriteOp newWriteOp(String stream,
+                       ByteBuffer data,
+                       Long checksum,
+                       boolean isRecordSet) {
+        return new WriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
+            serverConfig, dlsnVersion, checksum, isRecordSet, featureChecksumDisabled,
+            accessControlManager);
+    }
+
+    @VisibleForTesting
+    Future<List<Void>> closeStreams() {
+        return streamManager.closeStreams();
+    }
+
+    @VisibleForTesting
+    public DistributedLogNamespace getDistributedLogNamespace() {
+        return dlNamespace;
+    }
+
+    @VisibleForTesting
+    StreamManager getStreamManager() {
+        return streamManager;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
new file mode 100644
index 0000000..17b5ab3
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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;
+
+/**
+ * Implement handling for an unrecoverable error.
+ */
+public interface FatalErrorHandler {
+
+    /**
+     * This method is invoked when an unrecoverable error has occurred
+     * and no progress can be made. It should implement a shutdown routine.
+     */
+    void notifyFatalError();
+}


[26/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 0e2a152..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 033882f..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.service;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 05ee085..0000000
--- a/distributedlog-client/src/main/resources/findbugsExclude.xml
+++ /dev/null
@@ -1,23 +0,0 @@
-<!--
-    Licensed to the Apache Software Foundation (ASF) under one
-    or more contributor license agreements.  See the NOTICE file
-    distributed with this work for additional information
-    regarding copyright ownership.  The ASF licenses this file
-    to you under the Apache License, Version 2.0 (the
-    "License"); you may not use this file except in compliance
-    with the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
-//-->
-<FindBugsFilter>
-  <Match>
-    <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~org\.apache\.distributedlog\.thrift.*" />
-  </Match>
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index d7494de..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 86d1c11..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 8ef33bd..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index e38c2ed..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index ad1c878..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 6d9a471..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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());
-    }
-
-}


[23/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
new file mode 100644
index 0000000..b3f3368
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
new file mode 100644
index 0000000..ed6269b
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
new file mode 100644
index 0000000..d7e2c94
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
new file mode 100644
index 0000000..f3c24ca
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
new file mode 100644
index 0000000..486bd6f
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/package-info.java
new file mode 100644
index 0000000..d22b0da
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
new file mode 100644
index 0000000..9b5c7f6
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
new file mode 100644
index 0000000..769cca8
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
new file mode 100644
index 0000000..6ef1d8e
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
new file mode 100644
index 0000000..17b70be
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
new file mode 100644
index 0000000..0a6b076
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
new file mode 100644
index 0000000..4161afb
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java
new file mode 100644
index 0000000..2ac5be3
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
new file mode 100644
index 0000000..023799c
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
new file mode 100644
index 0000000..81cda2f
--- /dev/null
+++ b/distributedlog-proxy-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;


[14/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
new file mode 100644
index 0000000..c0c0972
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
@@ -0,0 +1,926 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.StatusCode;
+import org.apache.distributedlog.exceptions.StreamNotReadyException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.io.Abortables;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.service.FatalErrorHandler;
+import org.apache.distributedlog.service.ServerFeatureKeys;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.stream.limiter.StreamRequestLimiter;
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.stats.BroadCastStatsLogger;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.TimeSequencer;
+import org.apache.distributedlog.util.Utils;
+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.TimeoutException;
+import com.twitter.util.Timer;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+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.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.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Implementation of {@link Stream}.
+ */
+public class StreamImpl implements Stream {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamImpl.class);
+
+    /**
+     * The status of the stream.
+     *
+     * <p>The status change of the stream should just go in one direction. If a stream hits
+     * any error, the stream should be put in error state. If a stream is in error state,
+     * it should be removed and not reused anymore.
+     */
+    public enum StreamStatus {
+        UNINITIALIZED(-1),
+        INITIALIZING(0),
+        INITIALIZED(1),
+        CLOSING(-4),
+        CLOSED(-5),
+        // if a stream is in error state, it should be abort during closing.
+        ERROR(-6);
+
+        final int code;
+
+        StreamStatus(int code) {
+            this.code = code;
+        }
+
+        int getCode() {
+            return code;
+        }
+
+        public static boolean isUnavailable(StreamStatus status) {
+            return StreamStatus.ERROR == status || StreamStatus.CLOSING == status || StreamStatus.CLOSED == status;
+        }
+    }
+
+    private final String name;
+    private final Partition partition;
+    private DistributedLogManager manager;
+
+    private volatile AsyncLogWriter writer;
+    private volatile StreamStatus status;
+    private volatile String owner;
+    private volatile Throwable lastException;
+    private volatile Queue<StreamOp> pendingOps = new ArrayDeque<StreamOp>();
+
+    private final Promise<Void> closePromise = new Promise<Void>();
+    private final Object txnLock = new Object();
+    private final TimeSequencer sequencer = new TimeSequencer();
+    private final StreamRequestLimiter limiter;
+    private final DynamicDistributedLogConfiguration dynConf;
+    private final DistributedLogConfiguration dlConfig;
+    private final DistributedLogNamespace dlNamespace;
+    private final String clientId;
+    private final OrderedScheduler scheduler;
+    private final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
+    private final Feature featureRateLimitDisabled;
+    private final StreamManager streamManager;
+    private final StreamConfigProvider streamConfigProvider;
+    private final FatalErrorHandler fatalErrorHandler;
+    private final long streamProbationTimeoutMs;
+    private final long serviceTimeoutMs;
+    private final long writerCloseTimeoutMs;
+    private final boolean failFastOnStreamNotReady;
+    private final HashedWheelTimer requestTimer;
+    private final Timer futureTimer;
+
+    // Stats
+    private final StatsLogger streamLogger;
+    private final StatsLogger streamExceptionStatLogger;
+    private final StatsLogger limiterStatLogger;
+    private final Counter serviceTimeout;
+    private final OpStatsLogger streamAcquireStat;
+    private final OpStatsLogger writerCloseStatLogger;
+    private final Counter pendingOpsCounter;
+    private final Counter unexpectedExceptions;
+    private final Counter writerCloseTimeoutCounter;
+    private final StatsLogger exceptionStatLogger;
+    private final ConcurrentHashMap<String, Counter> exceptionCounters =
+        new ConcurrentHashMap<String, Counter>();
+    private final Gauge<Number> streamStatusGauge;
+
+    // Since we may create and discard streams at initialization if there's a race,
+    // must not do any expensive initialization here (particularly any locking or
+    // significant resource allocation etc.).
+    StreamImpl(final String name,
+               final Partition partition,
+               String clientId,
+               StreamManager streamManager,
+               StreamOpStats streamOpStats,
+               ServerConfiguration serverConfig,
+               DistributedLogConfiguration dlConfig,
+               DynamicDistributedLogConfiguration streamConf,
+               FeatureProvider featureProvider,
+               StreamConfigProvider streamConfigProvider,
+               DistributedLogNamespace dlNamespace,
+               OrderedScheduler scheduler,
+               FatalErrorHandler fatalErrorHandler,
+               HashedWheelTimer requestTimer,
+               Timer futureTimer) {
+        this.clientId = clientId;
+        this.dlConfig = dlConfig;
+        this.streamManager = streamManager;
+        this.name = name;
+        this.partition = partition;
+        this.status = StreamStatus.UNINITIALIZED;
+        this.lastException = new IOException("Fail to write record to stream " + name);
+        this.streamConfigProvider = streamConfigProvider;
+        this.dlNamespace = dlNamespace;
+        this.featureRateLimitDisabled = featureProvider.getFeature(
+            ServerFeatureKeys.SERVICE_RATE_LIMIT_DISABLED.name().toLowerCase());
+        this.scheduler = scheduler;
+        this.serviceTimeoutMs = serverConfig.getServiceTimeoutMs();
+        this.streamProbationTimeoutMs = serverConfig.getStreamProbationTimeoutMs();
+        this.writerCloseTimeoutMs = serverConfig.getWriterCloseTimeoutMs();
+        this.failFastOnStreamNotReady = dlConfig.getFailFastOnStreamNotReady();
+        this.fatalErrorHandler = fatalErrorHandler;
+        this.dynConf = streamConf;
+        StatsLogger limiterStatsLogger = BroadCastStatsLogger.two(
+            streamOpStats.baseScope("stream_limiter"),
+            streamOpStats.streamRequestScope(partition, "limiter"));
+        this.limiter = new StreamRequestLimiter(name, dynConf, limiterStatsLogger, featureRateLimitDisabled);
+        this.requestTimer = requestTimer;
+        this.futureTimer = futureTimer;
+
+        // Stats
+        this.streamLogger = streamOpStats.streamRequestStatsLogger(partition);
+        this.limiterStatLogger = streamOpStats.baseScope("request_limiter");
+        this.streamExceptionStatLogger = streamLogger.scope("exceptions");
+        this.serviceTimeout = streamOpStats.baseCounter("serviceTimeout");
+        StatsLogger streamsStatsLogger = streamOpStats.baseScope("streams");
+        this.streamAcquireStat = streamsStatsLogger.getOpStatsLogger("acquire");
+        this.pendingOpsCounter = streamOpStats.baseCounter("pending_ops");
+        this.unexpectedExceptions = streamOpStats.baseCounter("unexpected_exceptions");
+        this.exceptionStatLogger = streamOpStats.requestScope("exceptions");
+        this.writerCloseStatLogger = streamsStatsLogger.getOpStatsLogger("writer_close");
+        this.writerCloseTimeoutCounter = streamsStatsLogger.getCounter("writer_close_timeouts");
+        // Gauges
+        this.streamStatusGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return StreamStatus.UNINITIALIZED.getCode();
+            }
+            @Override
+            public Number getSample() {
+                return status.getCode();
+            }
+        };
+    }
+
+    @Override
+    public String getOwner() {
+        return owner;
+    }
+
+    @Override
+    public String getStreamName() {
+        return name;
+    }
+
+    @Override
+    public DynamicDistributedLogConfiguration getStreamConfiguration() {
+        return dynConf;
+    }
+
+    @Override
+    public Partition getPartition() {
+        return partition;
+    }
+
+    private DistributedLogManager openLog(String name) throws IOException {
+        Optional<DistributedLogConfiguration> dlConf = Optional.<DistributedLogConfiguration>absent();
+        Optional<DynamicDistributedLogConfiguration> dynDlConf = Optional.of(dynConf);
+        Optional<StatsLogger> perStreamStatsLogger = Optional.of(streamLogger);
+        return dlNamespace.openLog(name, dlConf, dynDlConf, perStreamStatsLogger);
+    }
+
+    // Expensive initialization, only called once per stream.
+    @Override
+    public void initialize() throws IOException {
+        manager = openLog(name);
+
+        // Better to avoid registering the gauge multiple times, so do this in init
+        // which only gets called once.
+        streamLogger.registerGauge("stream_status", this.streamStatusGauge);
+
+        // Signal initialization is complete, should be last in this method.
+        status = StreamStatus.INITIALIZING;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("Stream:%s, %s, %s Status:%s", name, manager, writer, status);
+    }
+
+    @Override
+    public void start() {
+        // acquire the stream
+        acquireStream().addEventListener(new FutureEventListener<Boolean>() {
+                @Override
+                public void onSuccess(Boolean success) {
+                    if (!success) {
+                        // failed to acquire the stream. set the stream in error status and close it.
+                        setStreamInErrorStatus();
+                        requestClose("Failed to acquire the ownership");
+                    }
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    // unhandled exceptions
+                    logger.error("Stream {} threw unhandled exception : ", name, cause);
+                    // failed to acquire the stream. set the stream in error status and close it.
+                    setStreamInErrorStatus();
+                    requestClose("Unhandled exception");
+                }
+            });
+    }
+
+    //
+    // Stats Operations
+    //
+
+    void countException(Throwable t, StatsLogger streamExceptionLogger) {
+        String exceptionName = null == t ? "null" : t.getClass().getName();
+        Counter counter = exceptionCounters.get(exceptionName);
+        if (null == counter) {
+            counter = exceptionStatLogger.getCounter(exceptionName);
+            Counter oldCounter = exceptionCounters.putIfAbsent(exceptionName, counter);
+            if (null != oldCounter) {
+                counter = oldCounter;
+            }
+        }
+        counter.inc();
+        streamExceptionLogger.getCounter(exceptionName).inc();
+    }
+
+    boolean isCriticalException(Throwable cause) {
+        return !(cause instanceof OwnershipAcquireFailedException);
+    }
+
+    //
+    // Service Timeout:
+    // - schedule a timeout function to handle operation timeouts: {@link #handleServiceTimeout(String)}
+    // - if the operation is completed within timeout period, cancel the timeout.
+    //
+
+    void scheduleTimeout(final StreamOp op) {
+        final Timeout timeout = requestTimer.newTimeout(new TimerTask() {
+            @Override
+            public void run(Timeout timeout) throws Exception {
+                if (!timeout.isCancelled()) {
+                    serviceTimeout.inc();
+                    handleServiceTimeout("Operation " + op.getClass().getName() + " timeout");
+                }
+            }
+        }, serviceTimeoutMs, TimeUnit.MILLISECONDS);
+        op.responseHeader().ensure(new Function0<BoxedUnit>() {
+            @Override
+            public BoxedUnit apply() {
+                timeout.cancel();
+                return null;
+            }
+        });
+    }
+
+    /**
+     * Close the stream and schedule cache eviction at some point in the future.
+     * We delay this as a way to place the stream in a probationary state--cached
+     * in the proxy but unusable.
+     * This mechanism helps the cluster adapt to situations where a proxy has
+     * persistent connectivity/availability issues, because it keeps an affected
+     * stream off the proxy for a period of time, hopefully long enough for the
+     * issues to be resolved, or for whoop to kick in and kill the shard.
+     */
+    void handleServiceTimeout(String reason) {
+        synchronized (this) {
+            if (StreamStatus.isUnavailable(status)) {
+                return;
+            }
+            // Mark stream in error state
+            setStreamInErrorStatus();
+        }
+
+        // Async close request, and schedule eviction when its done.
+        Future<Void> closeFuture = requestClose(reason, false /* dont remove */);
+        closeFuture.onSuccess(new AbstractFunction1<Void, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(Void result) {
+                streamManager.scheduleRemoval(StreamImpl.this, streamProbationTimeoutMs);
+                return BoxedUnit.UNIT;
+            }
+        });
+    }
+
+    //
+    // Submit the operation to the stream.
+    //
+
+    /**
+     * Execute the StreamOp. If reacquire is needed, this may initiate reacquire and queue the op for
+     * execution once complete.
+     *
+     * @param op
+     *          stream operation to execute.
+     */
+    @Override
+    public void submit(StreamOp op) {
+        try {
+            limiter.apply(op);
+        } catch (OverCapacityException ex) {
+            op.fail(ex);
+            return;
+        }
+
+        // Timeout stream op if requested.
+        if (serviceTimeoutMs > 0) {
+            scheduleTimeout(op);
+        }
+
+        boolean completeOpNow = false;
+        boolean success = true;
+        if (StreamStatus.isUnavailable(status)) {
+            // Stream is closed, fail the op immediately
+            op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
+            return;
+        } else if (StreamStatus.INITIALIZED == status && writer != null) {
+            completeOpNow = true;
+            success = true;
+        } else {
+            synchronized (this) {
+                if (StreamStatus.isUnavailable(status)) {
+                    // Stream is closed, fail the op immediately
+                    op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
+                    return;
+                } else if (StreamStatus.INITIALIZED == status) {
+                    completeOpNow = true;
+                    success = true;
+                } else if (failFastOnStreamNotReady) {
+                    op.fail(new StreamNotReadyException("Stream " + name + " is not ready; status = " + status));
+                    return;
+                } else { // the stream is still initializing
+                    pendingOps.add(op);
+                    pendingOpsCounter.inc();
+                    if (1 == pendingOps.size()) {
+                        if (op instanceof HeartbeatOp) {
+                            ((HeartbeatOp) op).setWriteControlRecord(true);
+                        }
+                    }
+                }
+            }
+        }
+        if (completeOpNow) {
+            executeOp(op, success);
+        }
+    }
+
+    //
+    // Execute operations and handle exceptions on operations
+    //
+
+    /**
+     * Execute the <i>op</i> immediately.
+     *
+     * @param op
+     *          stream operation to execute.
+     * @param success
+     *          whether the operation is success or not.
+     */
+    void executeOp(final StreamOp op, boolean success) {
+        final AsyncLogWriter writer;
+        final Throwable lastException;
+        synchronized (this) {
+            writer = this.writer;
+            lastException = this.lastException;
+        }
+        if (null != writer && success) {
+            op.execute(writer, sequencer, txnLock)
+                    .addEventListener(new FutureEventListener<Void>() {
+                @Override
+                public void onSuccess(Void value) {
+                    // nop
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    boolean countAsException = true;
+                    if (cause instanceof DLException) {
+                        final DLException dle = (DLException) cause;
+                        switch (dle.getCode()) {
+                        case StatusCode.FOUND:
+                            assert(cause instanceof OwnershipAcquireFailedException);
+                            countAsException = false;
+                            handleExceptionOnStreamOp(op, cause);
+                            break;
+                        case StatusCode.ALREADY_CLOSED:
+                            assert(cause instanceof AlreadyClosedException);
+                            op.fail(cause);
+                            handleAlreadyClosedException((AlreadyClosedException) cause);
+                            break;
+                        // exceptions that mostly from client (e.g. too large record)
+                        case StatusCode.NOT_IMPLEMENTED:
+                        case StatusCode.METADATA_EXCEPTION:
+                        case StatusCode.LOG_EMPTY:
+                        case StatusCode.LOG_NOT_FOUND:
+                        case StatusCode.TRUNCATED_TRANSACTION:
+                        case StatusCode.END_OF_STREAM:
+                        case StatusCode.TRANSACTION_OUT_OF_ORDER:
+                        case StatusCode.INVALID_STREAM_NAME:
+                        case StatusCode.TOO_LARGE_RECORD:
+                        case StatusCode.STREAM_NOT_READY:
+                        case StatusCode.OVER_CAPACITY:
+                            op.fail(cause);
+                            break;
+                        // the DL writer hits exception, simple set the stream to error status
+                        // and fail the request
+                        default:
+                            handleExceptionOnStreamOp(op, cause);
+                            break;
+                        }
+                    } else {
+                        handleExceptionOnStreamOp(op, cause);
+                    }
+                    if (countAsException) {
+                        countException(cause, streamExceptionStatLogger);
+                    }
+                }
+            });
+        } else {
+            if (null != lastException) {
+                op.fail(lastException);
+            } else {
+                op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
+            }
+        }
+    }
+
+    /**
+     * Handle exception when executing <i>op</i>.
+     *
+     * @param op
+     *          stream operation executing
+     * @param cause
+     *          exception received when executing <i>op</i>
+     */
+    private void handleExceptionOnStreamOp(StreamOp op, final Throwable cause) {
+        AsyncLogWriter oldWriter = null;
+        boolean statusChanged = false;
+        synchronized (this) {
+            if (StreamStatus.INITIALIZED == status) {
+                oldWriter = setStreamStatus(StreamStatus.ERROR, StreamStatus.INITIALIZED, null, cause);
+                statusChanged = true;
+            }
+        }
+        if (statusChanged) {
+            Abortables.asyncAbort(oldWriter, false);
+            if (isCriticalException(cause)) {
+                logger.error("Failed to write data into stream {} : ", name, cause);
+            } else {
+                logger.warn("Failed to write data into stream {} : {}", name, cause.getMessage());
+            }
+            requestClose("Failed to write data into stream " + name + " : " + cause.getMessage());
+        }
+        op.fail(cause);
+    }
+
+    /**
+     * Handling already closed exception.
+     */
+    private void handleAlreadyClosedException(AlreadyClosedException ace) {
+        unexpectedExceptions.inc();
+        logger.error("Encountered unexpected exception when writing data into stream {} : ", name, ace);
+        fatalErrorHandler.notifyFatalError();
+    }
+
+    //
+    // Acquire streams
+    //
+
+    Future<Boolean> acquireStream() {
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        final Promise<Boolean> acquirePromise = new Promise<Boolean>();
+        manager.openAsyncLogWriter().addEventListener(
+            FutureUtils.OrderedFutureEventListener.of(new FutureEventListener<AsyncLogWriter>() {
+
+            @Override
+            public void onSuccess(AsyncLogWriter w) {
+                onAcquireStreamSuccess(w, stopwatch, acquirePromise);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                onAcquireStreamFailure(cause, stopwatch, acquirePromise);
+            }
+
+        }, scheduler, getStreamName()));
+        return acquirePromise;
+    }
+
+    private void onAcquireStreamSuccess(AsyncLogWriter w,
+                                        Stopwatch stopwatch,
+                                        Promise<Boolean> acquirePromise) {
+        synchronized (txnLock) {
+            sequencer.setLastId(w.getLastTxId());
+        }
+        AsyncLogWriter oldWriter;
+        Queue<StreamOp> oldPendingOps;
+        boolean success;
+        synchronized (StreamImpl.this) {
+            oldWriter = setStreamStatus(StreamStatus.INITIALIZED,
+                    StreamStatus.INITIALIZING, w, null);
+            oldPendingOps = pendingOps;
+            pendingOps = new ArrayDeque<StreamOp>();
+            success = true;
+        }
+        // check if the stream is allowed to be acquired
+        if (!streamManager.allowAcquire(StreamImpl.this)) {
+            if (null != oldWriter) {
+                Abortables.asyncAbort(oldWriter, true);
+            }
+            int maxAcquiredPartitions = dynConf.getMaxAcquiredPartitionsPerProxy();
+            StreamUnavailableException sue = new StreamUnavailableException("Stream " + partition.getStream()
+                    + " is not allowed to acquire more than " + maxAcquiredPartitions + " partitions");
+            countException(sue, exceptionStatLogger);
+            logger.error("Failed to acquire stream {} because it is unavailable : {}",
+                    name, sue.getMessage());
+            synchronized (this) {
+                oldWriter = setStreamStatus(StreamStatus.ERROR,
+                        StreamStatus.INITIALIZED, null, sue);
+                // we don't switch the pending ops since they are already switched
+                // when setting the status to initialized
+                success = false;
+            }
+        }
+        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
+    }
+
+    private void onAcquireStreamFailure(Throwable cause,
+                                        Stopwatch stopwatch,
+                                        Promise<Boolean> acquirePromise) {
+        AsyncLogWriter oldWriter;
+        Queue<StreamOp> oldPendingOps;
+        boolean success;
+        if (cause instanceof AlreadyClosedException) {
+            countException(cause, streamExceptionStatLogger);
+            handleAlreadyClosedException((AlreadyClosedException) cause);
+            return;
+        } else {
+            if (isCriticalException(cause)) {
+                countException(cause, streamExceptionStatLogger);
+                logger.error("Failed to acquire stream {} : ", name, cause);
+            } else {
+                logger.warn("Failed to acquire stream {} : {}", name, cause.getMessage());
+            }
+            synchronized (StreamImpl.this) {
+                oldWriter = setStreamStatus(StreamStatus.ERROR,
+                        StreamStatus.INITIALIZING, null, cause);
+                oldPendingOps = pendingOps;
+                pendingOps = new ArrayDeque<StreamOp>();
+                success = false;
+            }
+        }
+        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
+    }
+
+    /**
+     * Process the pending request after acquired stream.
+     *
+     * @param success whether the acquisition succeed or not
+     * @param oldWriter the old writer to abort
+     * @param oldPendingOps the old pending ops to execute
+     * @param stopwatch stopwatch to measure the time spent on acquisition
+     * @param acquirePromise the promise to complete the acquire operation
+     */
+    void processPendingRequestsAfterAcquire(boolean success,
+                                            AsyncLogWriter oldWriter,
+                                            Queue<StreamOp> oldPendingOps,
+                                            Stopwatch stopwatch,
+                                            Promise<Boolean> acquirePromise) {
+        if (success) {
+            streamAcquireStat.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+        } else {
+            streamAcquireStat.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+        }
+        for (StreamOp op : oldPendingOps) {
+            executeOp(op, success);
+            pendingOpsCounter.dec();
+        }
+        Abortables.asyncAbort(oldWriter, true);
+        FutureUtils.setValue(acquirePromise, success);
+    }
+
+    //
+    // Stream Status Changes
+    //
+
+    synchronized void setStreamInErrorStatus() {
+        if (StreamStatus.CLOSING == status || StreamStatus.CLOSED == status) {
+            return;
+        }
+        this.status = StreamStatus.ERROR;
+    }
+
+    /**
+     * Update the stream status. The changes are only applied when there isn't status changed.
+     *
+     * @param newStatus
+     *          new status
+     * @param oldStatus
+     *          old status
+     * @param writer
+     *          new log writer
+     * @param t
+     *          new exception
+     * @return old writer if it exists
+     */
+    synchronized AsyncLogWriter setStreamStatus(StreamStatus newStatus,
+                                                StreamStatus oldStatus,
+                                                AsyncLogWriter writer,
+                                                Throwable t) {
+        if (oldStatus != this.status) {
+            logger.info("Stream {} status already changed from {} -> {} when trying to change it to {}",
+                    new Object[] { name, oldStatus, this.status, newStatus });
+            return null;
+        }
+
+        String owner = null;
+        if (t instanceof OwnershipAcquireFailedException) {
+            owner = ((OwnershipAcquireFailedException) t).getCurrentOwner();
+        }
+
+        AsyncLogWriter oldWriter = this.writer;
+        this.writer = writer;
+        if (null != owner && owner.equals(clientId)) {
+            unexpectedExceptions.inc();
+            logger.error("I am waiting myself {} to release lock on stream {}, so have to shut myself down :",
+                         new Object[] { owner, name, t });
+            // I lost the ownership but left a lock over zookeeper
+            // I should not ask client to redirect to myself again as I can't handle it :(
+            // shutdown myself
+            fatalErrorHandler.notifyFatalError();
+            this.owner = null;
+        } else {
+            this.owner = owner;
+        }
+        this.lastException = t;
+        this.status = newStatus;
+        if (StreamStatus.INITIALIZED == newStatus) {
+            streamManager.notifyAcquired(this);
+            logger.info("Inserted acquired stream {} -> writer {}", name, this);
+        } else {
+            streamManager.notifyReleased(this);
+            logger.info("Removed acquired stream {} -> writer {}", name, this);
+        }
+        return oldWriter;
+    }
+
+    //
+    // Stream Close Functions
+    //
+
+    void close(DistributedLogManager dlm) {
+        if (null != dlm) {
+            try {
+                dlm.close();
+            } catch (IOException ioe) {
+                logger.warn("Failed to close dlm for {} : ", name, ioe);
+            }
+        }
+    }
+
+    @Override
+    public Future<Void> requestClose(String reason) {
+        return requestClose(reason, true);
+    }
+
+    Future<Void> requestClose(String reason, boolean uncache) {
+        final boolean abort;
+        closeLock.writeLock().lock();
+        try {
+            if (StreamStatus.CLOSING == status
+                || StreamStatus.CLOSED == status) {
+                return closePromise;
+            }
+            logger.info("Request to close stream {} : {}", getStreamName(), reason);
+            // if the stream isn't closed from INITIALIZED state, we abort the stream instead of closing it.
+            abort = StreamStatus.INITIALIZED != status;
+            status = StreamStatus.CLOSING;
+            streamManager.notifyReleased(this);
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+        // we will fail the requests that are coming in between closing and closed only
+        // after the async writer is closed. so we could clear up the lock before redirect
+        // them.
+        close(abort, uncache);
+        return closePromise;
+    }
+
+    @Override
+    public void delete() throws IOException {
+        if (null != writer) {
+            Utils.close(writer);
+            synchronized (this) {
+                writer = null;
+                lastException = new StreamUnavailableException("Stream was deleted");
+            }
+        }
+        if (null == manager) {
+            throw new UnexpectedException("No stream " + name + " to delete");
+        }
+        manager.delete();
+    }
+
+    /**
+     * Post action executed after closing.
+     */
+    private void postClose(boolean uncache) {
+        closeManagerAndErrorOutPendingRequests();
+        unregisterGauge();
+        if (uncache) {
+            if (null != owner) {
+                long probationTimeoutMs = 2 * dlConfig.getZKSessionTimeoutMilliseconds() / 3;
+                streamManager.scheduleRemoval(this, probationTimeoutMs);
+            } else {
+                streamManager.notifyRemoved(this);
+                logger.info("Removed cached stream {}.", getStreamName());
+            }
+        }
+        FutureUtils.setValue(closePromise, null);
+    }
+
+    /**
+     * Shouldn't call close directly. The callers should call #requestClose instead
+     *
+     * @param shouldAbort shall we abort the stream instead of closing
+     */
+    private Future<Void> close(boolean shouldAbort, final boolean uncache) {
+        boolean abort;
+        closeLock.writeLock().lock();
+        try {
+            if (StreamStatus.CLOSED == status) {
+                return closePromise;
+            }
+            abort = shouldAbort || (StreamStatus.INITIALIZED != status && StreamStatus.CLOSING != status);
+            status = StreamStatus.CLOSED;
+            streamManager.notifyReleased(this);
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+        logger.info("Closing stream {} ...", name);
+        // Close the writers to release the locks before failing the requests
+        Future<Void> closeWriterFuture;
+        if (abort) {
+            closeWriterFuture = Abortables.asyncAbort(writer, true);
+        } else {
+            closeWriterFuture = Utils.asyncClose(writer, true);
+        }
+        // close the manager and error out pending requests after close writer
+        Duration closeWaitDuration;
+        if (writerCloseTimeoutMs <= 0) {
+            closeWaitDuration = Duration.Top();
+        } else {
+            closeWaitDuration = Duration.fromMilliseconds(writerCloseTimeoutMs);
+        }
+
+        FutureUtils.stats(
+                closeWriterFuture,
+                writerCloseStatLogger,
+                Stopwatch.createStarted()
+        ).masked().within(futureTimer, closeWaitDuration)
+                .addEventListener(FutureUtils.OrderedFutureEventListener.of(
+                new FutureEventListener<Void>() {
+                    @Override
+                    public void onSuccess(Void value) {
+                        postClose(uncache);
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        if (cause instanceof TimeoutException) {
+                            writerCloseTimeoutCounter.inc();
+                        }
+                        postClose(uncache);
+                    }
+                }, scheduler, name));
+        return closePromise;
+    }
+
+    private void closeManagerAndErrorOutPendingRequests() {
+        close(manager);
+        // Failed the pending requests.
+        Queue<StreamOp> oldPendingOps;
+        synchronized (this) {
+            oldPendingOps = pendingOps;
+            pendingOps = new ArrayDeque<StreamOp>();
+        }
+        StreamUnavailableException closingException =
+                new StreamUnavailableException("Stream " + name + " is closed.");
+        for (StreamOp op : oldPendingOps) {
+            op.fail(closingException);
+            pendingOpsCounter.dec();
+        }
+        limiter.close();
+        logger.info("Closed stream {}.", name);
+    }
+
+    /**
+     * clean up the gauge to help GC.
+     */
+    private void unregisterGauge(){
+        streamLogger.unregisterGauge("stream_status", this.streamStatusGauge);
+    }
+
+    // Test-only apis
+
+    @VisibleForTesting
+    public int numPendingOps() {
+        Queue<StreamOp> queue = pendingOps;
+        return null == queue ? 0 : queue.size();
+    }
+
+    @VisibleForTesting
+    public StreamStatus getStatus() {
+        return status;
+    }
+
+    @VisibleForTesting
+    public void setStatus(StreamStatus status) {
+        this.status = status;
+    }
+
+    @VisibleForTesting
+    public AsyncLogWriter getWriter() {
+        return writer;
+    }
+
+    @VisibleForTesting
+    public DistributedLogManager getManager() {
+        return manager;
+    }
+
+    @VisibleForTesting
+    public Throwable getLastException() {
+        return lastException;
+    }
+
+    @VisibleForTesting
+    public Future<Void> getCloseFuture() {
+        return closePromise;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
new file mode 100644
index 0000000..d86c538
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import com.google.common.base.Optional;
+import com.twitter.util.Future;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Manage lifecycle of streams.
+ *
+ * <p>StreamManager is responsible for creating, destroying, and keeping track of Stream objects.
+ *
+ * <p>Stream objects, which are managed by StreamManager and created by StreamFactory, are essentially the
+ * per stream request handlers, responsible fo dispatching ex. write requests to an underlying AsyncLogWriter,
+ * managing stream lock, interpreting exceptions, error conditions, and etc.
+ */
+public interface StreamManager {
+
+    /**
+     * Get a cached stream, returning null if it doesnt exist.
+     * @param stream name
+     * @return the cached stream
+     */
+    Stream getStream(String stream);
+
+    /**
+     * Get a cached stream and create a new one if it doesnt exist.
+     * @param streamName stream name
+     * @param start whether to start the stream after it is created.
+     * @return future satisfied once close complete
+     */
+    Stream getOrCreateStream(String streamName, boolean start) throws IOException;
+
+    /**
+     * Asynchronously create a new stream.
+     * @param stream
+     * @return Future satisfied once the stream is created
+     */
+    Future<Void> createStreamAsync(String stream);
+
+    /**
+     * Is acquiring stream allowed?
+     *
+     * @param stream
+     *          stream instance
+     * @return true if it is allowed to acquire this stream, otherwise false.
+     */
+    boolean allowAcquire(Stream stream);
+
+    /**
+     * Notify the manager that a stream was acquired.
+     * @param stream being acquired
+     */
+    void notifyAcquired(Stream stream);
+
+    /**
+     * Notify the manager that a stream was released.
+     * @param stream being released
+     */
+    void notifyReleased(Stream stream);
+
+    /**
+     * Notify the manager that a stream was completely removed.
+     * @param stream being uncached
+     * @return whether the stream existed or not
+     */
+    boolean notifyRemoved(Stream stream);
+
+    /**
+     * Asynchronous delete method.
+     * @param streamName stream name
+     * @return future satisfied once delete complete
+     */
+    Future<Void> deleteAndRemoveAsync(String streamName);
+
+    /**
+     * Asynchronous close and uncache method.
+     * @param streamName stream name
+     * @return future satisfied once close and uncache complete
+     */
+    Future<Void> closeAndRemoveAsync(String streamName);
+
+    /**
+     * Close and uncache after delayMs.
+     * @param stream to remove
+     */
+    void scheduleRemoval(Stream stream, long delayMs);
+
+    /**
+     * Close all stream.
+     * @return future satisfied all streams closed
+     */
+    Future<List<Void>> closeStreams();
+
+    /**
+     * Return map with stream ownership info.
+     * @param regex for filtering streams
+     * @return map containing ownership info
+     */
+    Map<String, String> getStreamOwnershipMap(Optional<String> regex);
+
+    /**
+     * Number of acquired streams.
+     * @return number of acquired streams
+     */
+    int numAcquired();
+
+    /**
+     * Number of cached streams.
+     * @return number of cached streams
+     */
+    int numCached();
+
+    /**
+     * Is the stream denoted by streamName in the acquired state.
+     * @return true if the stream is in the acquired state
+     */
+    boolean isAcquired(String streamName);
+
+    /**
+     * Close manager and disallow further activity.
+     */
+    void close();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
new file mode 100644
index 0000000..5d54738
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
@@ -0,0 +1,413 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.ServiceUnavailableException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.service.streamset.PartitionMap;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.util.ConfUtils;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * StreamManagerImpl is the default implementation responsible for creating, destroying, and keeping track
+ * of Streams.
+ *
+ * <p>StreamFactory, supplied to StreamManagerImpl in the constructor below, is reposible simply for creating
+ * a stream object in isolation from the rest of the system. We pass a StreamFactory in instead of simply
+ * creating StreamImpl's ourselves in order to inject dependencies without bloating the StreamManagerImpl
+ * constructor.
+ */
+public class StreamManagerImpl implements StreamManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamManagerImpl.class);
+
+    private final ConcurrentHashMap<String, Stream> streams =
+        new ConcurrentHashMap<String, Stream>();
+    private final AtomicInteger numCached = new AtomicInteger(0);
+
+    private final ConcurrentHashMap<String, Stream> acquiredStreams =
+        new ConcurrentHashMap<String, Stream>();
+    private final AtomicInteger numAcquired = new AtomicInteger(0);
+
+    //
+    // Partitions
+    //
+    private final StreamPartitionConverter partitionConverter;
+    private final PartitionMap cachedPartitions = new PartitionMap();
+    private final PartitionMap acquiredPartitions = new PartitionMap();
+
+    final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
+    private final ScheduledExecutorService executorService;
+    private final DistributedLogConfiguration dlConfig;
+    private final StreamConfigProvider streamConfigProvider;
+    private final String clientId;
+    private boolean closed = false;
+    private final StreamFactory streamFactory;
+    private final DistributedLogNamespace dlNamespace;
+
+    public StreamManagerImpl(String clientId,
+                             DistributedLogConfiguration dlConfig,
+                             ScheduledExecutorService executorService,
+                             StreamFactory streamFactory,
+                             StreamPartitionConverter partitionConverter,
+                             StreamConfigProvider streamConfigProvider,
+                             DistributedLogNamespace dlNamespace) {
+        this.clientId = clientId;
+        this.executorService = executorService;
+        this.streamFactory = streamFactory;
+        this.partitionConverter = partitionConverter;
+        this.dlConfig = dlConfig;
+        this.streamConfigProvider = streamConfigProvider;
+        this.dlNamespace = dlNamespace;
+    }
+
+    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
+        Optional<DynamicDistributedLogConfiguration> dynDlConf =
+                streamConfigProvider.getDynamicStreamConfig(streamName);
+        if (dynDlConf.isPresent()) {
+            return dynDlConf.get();
+        } else {
+            return ConfUtils.getConstDynConf(dlConfig);
+        }
+    }
+
+    @Override
+    public boolean allowAcquire(Stream stream) {
+        return acquiredPartitions.addPartition(
+                stream.getPartition(),
+                stream.getStreamConfiguration().getMaxAcquiredPartitionsPerProxy());
+    }
+
+    /**
+     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
+     * try to acquire the same read-write lock).
+     */
+    @Override
+    public Future<Void> deleteAndRemoveAsync(final String stream) {
+        final Promise<Void> result = new Promise<Void>();
+        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
+            @Override
+            public void run() {
+                result.become(doDeleteAndRemoveAsync(stream));
+            }
+        }, 0);
+        if (null == scheduleFuture) {
+            return Future.exception(
+                new ServiceUnavailableException("Couldn't schedule a delete task."));
+        }
+        return result;
+    }
+
+    /**
+     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
+     * try to acquire the same read-write lock).
+     */
+    @Override
+    public Future<Void> closeAndRemoveAsync(final String streamName) {
+        final Promise<Void> releasePromise = new Promise<Void>();
+        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
+            @Override
+            public void run() {
+                releasePromise.become(doCloseAndRemoveAsync(streamName));
+            }
+        }, 0);
+        if (null == scheduleFuture) {
+            return Future.exception(
+                new ServiceUnavailableException("Couldn't schedule a release task."));
+        }
+        return releasePromise;
+    }
+
+    @Override
+    public Future<Void> createStreamAsync(final String stream) {
+        final Promise<Void> createPromise = new Promise<Void>();
+        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    dlNamespace.createLog(stream);
+                    createPromise.setValue(null);
+                } catch (Exception e) {
+                    createPromise.setException(e);
+                }
+            }
+        }, 0);
+        if (null == scheduleFuture) {
+            return Future.exception(
+                new ServiceUnavailableException("Couldn't schedule a create task."));
+        }
+        return createPromise;
+    }
+
+    @Override
+    public void notifyReleased(Stream stream) {
+        acquiredPartitions.removePartition(stream.getPartition());
+        if (acquiredStreams.remove(stream.getStreamName(), stream)) {
+            numAcquired.getAndDecrement();
+        }
+    }
+
+    @Override
+    public void notifyAcquired(Stream stream) {
+        if (null == acquiredStreams.put(stream.getStreamName(), stream)) {
+            numAcquired.getAndIncrement();
+        }
+    }
+
+    @Override
+    public boolean notifyRemoved(Stream stream) {
+        cachedPartitions.removePartition(stream.getPartition());
+        if (streams.remove(stream.getStreamName(), stream)) {
+            numCached.getAndDecrement();
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getStreamOwnershipMap(Optional<String> regex) {
+        Map<String, String> ownershipMap = new HashMap<String, String>();
+        for (Map.Entry<String, Stream> entry : acquiredStreams.entrySet()) {
+            String name = entry.getKey();
+            if (regex.isPresent() && !name.matches(regex.get())) {
+                continue;
+            }
+            Stream stream = entry.getValue();
+            if (null == stream) {
+                continue;
+            }
+            String owner = stream.getOwner();
+            if (null == owner) {
+                ownershipMap.put(name, clientId);
+            }
+        }
+        return ownershipMap;
+    }
+
+    @Override
+    public Stream getStream(String stream) {
+        return streams.get(stream);
+    }
+
+    @Override
+    public Stream getOrCreateStream(String streamName, boolean start) throws IOException {
+        Stream stream = streams.get(streamName);
+        if (null == stream) {
+            closeLock.readLock().lock();
+            try {
+                if (closed) {
+                    return null;
+                }
+                DynamicDistributedLogConfiguration dynConf = getDynConf(streamName);
+                int maxCachedPartitions = dynConf.getMaxCachedPartitionsPerProxy();
+
+                // get partition from the stream name
+                Partition partition = partitionConverter.convert(streamName);
+
+                // add partition to cached map
+                if (!cachedPartitions.addPartition(partition, maxCachedPartitions)) {
+                    throw new StreamUnavailableException("Stream " + streamName
+                            + " is not allowed to cache more than " + maxCachedPartitions + " partitions");
+                }
+
+                stream = newStream(streamName, dynConf);
+                Stream oldWriter = streams.putIfAbsent(streamName, stream);
+                if (null != oldWriter) {
+                    stream = oldWriter;
+                } else {
+                    numCached.getAndIncrement();
+                    logger.info("Inserted mapping stream name {} -> stream {}", streamName, stream);
+                    stream.initialize();
+                    if (start) {
+                        stream.start();
+                    }
+                }
+            } finally {
+                closeLock.readLock().unlock();
+            }
+        }
+        return stream;
+    }
+
+    @Override
+    public Future<List<Void>> closeStreams() {
+        int numAcquired = acquiredStreams.size();
+        int numCached = streams.size();
+        logger.info("Closing all acquired streams : acquired = {}, cached = {}.",
+            numAcquired, numCached);
+        Set<Stream> streamsToClose = new HashSet<Stream>();
+        streamsToClose.addAll(streams.values());
+        return closeStreams(streamsToClose, Optional.<RateLimiter>absent());
+    }
+
+    @Override
+    public void scheduleRemoval(final Stream stream, long delayMs) {
+        if (delayMs > 0) {
+            logger.info("Scheduling removal of stream {} from cache after {} sec.",
+                    stream.getStreamName(), delayMs);
+        }
+        schedule(new Runnable() {
+            @Override
+            public void run() {
+                if (notifyRemoved(stream)) {
+                    logger.info("Removed cached stream {} after probation.", stream.getStreamName());
+                } else {
+                    logger.info("Cached stream {} already removed.", stream.getStreamName());
+                }
+            }
+        }, delayMs);
+    }
+
+    @Override
+    public int numAcquired() {
+        return numAcquired.get();
+    }
+
+    @Override
+    public int numCached() {
+        return numCached.get();
+    }
+
+    @Override
+    public boolean isAcquired(String streamName) {
+        return acquiredStreams.containsKey(streamName);
+    }
+
+    @Override
+    public void close() {
+        closeLock.writeLock().lock();
+        try {
+            if (closed) {
+                return;
+            }
+            closed = true;
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+    }
+
+    private Future<List<Void>> closeStreams(Set<Stream> streamsToClose, Optional<RateLimiter> rateLimiter) {
+        if (streamsToClose.isEmpty()) {
+            logger.info("No streams to close.");
+            List<Void> emptyList = new ArrayList<Void>();
+            return Future.value(emptyList);
+        }
+        List<Future<Void>> futures = new ArrayList<Future<Void>>(streamsToClose.size());
+        for (Stream stream : streamsToClose) {
+            if (rateLimiter.isPresent()) {
+                rateLimiter.get().acquire();
+            }
+            futures.add(stream.requestClose("Close Streams"));
+        }
+        return Future.collect(futures);
+    }
+
+    private Stream newStream(String name, DynamicDistributedLogConfiguration streamConf) {
+        return streamFactory.create(name, streamConf, this);
+    }
+
+    public Future<Void> doCloseAndRemoveAsync(final String streamName) {
+        Stream stream = streams.get(streamName);
+        if (null == stream) {
+            logger.info("No stream {} to release.", streamName);
+            return Future.value(null);
+        } else {
+            return stream.requestClose("release ownership");
+        }
+    }
+
+    /**
+     * Dont schedule if we're closed - closeLock is acquired to close, so if we acquire the
+     * lock and discover we're not closed, we won't schedule.
+     */
+    private java.util.concurrent.Future<?> schedule(Runnable runnable, long delayMs) {
+        closeLock.readLock().lock();
+        try {
+            if (closed) {
+                return null;
+            } else if (delayMs > 0) {
+                return executorService.schedule(runnable, delayMs, TimeUnit.MILLISECONDS);
+            } else {
+                return executorService.submit(runnable);
+            }
+        } catch (RejectedExecutionException ree) {
+            logger.error("Failed to schedule task {} in {} ms : ",
+                    new Object[] { runnable, delayMs, ree });
+            return null;
+        } finally {
+            closeLock.readLock().unlock();
+        }
+    }
+
+    private Future<Void> doDeleteAndRemoveAsync(final String streamName) {
+        Stream stream = streams.get(streamName);
+        if (null == stream) {
+            logger.warn("No stream {} to delete.", streamName);
+            return Future.exception(new UnexpectedException("No stream " + streamName + " to delete."));
+        } else {
+            Future<Void> result;
+            logger.info("Deleting stream {}, {}", streamName, stream);
+            try {
+                stream.delete();
+                result = stream.requestClose("Stream Deleted");
+            } catch (IOException e) {
+                logger.error("Failed on removing stream {} : ", streamName, e);
+                result = Future.exception(e);
+            }
+            return result;
+        }
+    }
+
+    @VisibleForTesting
+    public ConcurrentHashMap<String, Stream> getCachedStreams() {
+        return streams;
+    }
+
+    @VisibleForTesting
+    public ConcurrentHashMap<String, Stream> getAcquiredStreams() {
+        return acquiredStreams;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
new file mode 100644
index 0000000..d0b8de4
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+
+/**
+ * An operation applied to a stream.
+ */
+public interface StreamOp {
+    /**
+     * Execute a stream op with the supplied writer.
+     *
+     * @param writer active writer for applying the change
+     * @param sequencer sequencer used for generating transaction id for stream operations
+     * @param txnLock transaction lock to guarantee ordering of transaction id
+     * @return a future satisfied when the operation completes execution
+     */
+    Future<Void> execute(AsyncLogWriter writer,
+                         Sequencer sequencer,
+                         Object txnLock);
+
+    /**
+     * Invoked before the stream op is executed.
+     */
+    void preExecute() throws DLException;
+
+    /**
+     * Return the response header (containing the status code etc.).
+     *
+     * @return A future containing the response header or the exception
+     *      encountered by the op if it failed.
+     */
+    Future<ResponseHeader> responseHeader();
+
+    /**
+     * Abort the operation with the givem exception.
+     */
+    void fail(Throwable t);
+
+    /**
+     * Return the stream name.
+     */
+    String streamName();
+
+    /**
+     * Stopwatch gives the start time of the operation.
+     */
+    Stopwatch stopwatch();
+
+    /**
+     * Compute checksum from arguments.
+     */
+    Long computeChecksum();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
new file mode 100644
index 0000000..f3fc610
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.stats.BroadCastStatsLogger;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Encapsulate stream op stats construction to make it easier to access stream
+ * op stats consistently from different scopes.
+ */
+public class StreamOpStats {
+    private final StatsLogger baseStatsLogger;
+    private final StatsLogger requestStatsLogger;
+    private final StatsLogger recordsStatsLogger;
+    private final StatsLogger requestDeniedStatsLogger;
+    private final StatsLogger streamStatsLogger;
+
+    public StreamOpStats(StatsLogger statsLogger,
+                         StatsLogger perStreamStatsLogger) {
+        this.baseStatsLogger = statsLogger;
+        this.requestStatsLogger = statsLogger.scope("request");
+        this.recordsStatsLogger = statsLogger.scope("records");
+        this.requestDeniedStatsLogger = statsLogger.scope("denied");
+        this.streamStatsLogger = perStreamStatsLogger;
+    }
+
+    public StatsLogger baseStatsLogger(String opName) {
+        return baseStatsLogger;
+    }
+
+    public Counter baseCounter(String opName) {
+        return baseStatsLogger.getCounter(opName);
+    }
+
+    public StatsLogger baseScope(String opName) {
+        return baseStatsLogger.scope(opName);
+    }
+
+    public OpStatsLogger requestLatencyStat(String opName) {
+        return requestStatsLogger.getOpStatsLogger(opName);
+    }
+
+    public StatsLogger requestScope(String scopeName) {
+        return requestStatsLogger.scope(scopeName);
+    }
+
+    public Counter scopedRequestCounter(String opName, String counterName) {
+        return requestScope(opName).getCounter(counterName);
+    }
+
+    public Counter requestCounter(String counterName) {
+        return requestStatsLogger.getCounter(counterName);
+    }
+
+    public Counter requestPendingCounter(String counterName) {
+        return requestCounter(counterName);
+    }
+
+    public Counter requestDeniedCounter(String counterName) {
+        return requestDeniedStatsLogger.getCounter(counterName);
+    }
+
+    public Counter recordsCounter(String counterName) {
+        return recordsStatsLogger.getCounter(counterName);
+    }
+
+    public StatsLogger streamRequestStatsLogger(Partition partition) {
+        return BroadCastStatsLogger.masterslave(
+            streamStatsLogger.scope(partition.getStream()).scope("partition")
+                .scope(partition.getPaddedId()), streamStatsLogger.scope(partition.getStream())
+                .scope("aggregate"));
+    }
+
+    public StatsLogger streamRequestScope(Partition partition, String scopeName) {
+        return streamRequestStatsLogger(partition).scope(scopeName);
+    }
+
+    public OpStatsLogger streamRequestLatencyStat(Partition partition, String opName) {
+        return streamRequestStatsLogger(partition).getOpStatsLogger(opName);
+    }
+
+    public Counter streamRequestCounter(Partition partition, String opName, String counterName) {
+        return streamRequestScope(partition, opName).getCounter(counterName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
new file mode 100644
index 0000000..0036a5c
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.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.service.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.RequestDeniedException;
+import org.apache.distributedlog.protocol.util.ProtocolUtils;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to truncate a log stream.
+ */
+public class TruncateOp extends AbstractWriteOp {
+
+    private static final Logger logger = LoggerFactory.getLogger(TruncateOp.class);
+
+    private final Counter deniedTruncateCounter;
+    private final DLSN dlsn;
+    private final AccessControlManager accessControlManager;
+
+    public TruncateOp(String stream,
+                      DLSN dlsn,
+                      StatsLogger statsLogger,
+                      StatsLogger perStreamStatsLogger,
+                      Long checksum,
+                      Feature checksumDisabledFeature,
+                      AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "truncate"), checksum, checksumDisabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedTruncateCounter = streamOpStats.requestDeniedCounter("truncate");
+        this.accessControlManager = accessControlManager;
+        this.dlsn = dlsn;
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return ProtocolUtils.truncateOpCRC32(stream, dlsn);
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        if (!stream.equals(writer.getStreamName())) {
+            logger.error("Truncate: Stream Name Mismatch in the Stream Map {}, {}", stream, writer.getStreamName());
+            return Future.exception(new IllegalStateException("The stream mapping is incorrect, fail the request"));
+        }
+        return writer.truncate(dlsn).map(new AbstractFunction1<Boolean, WriteResponse>() {
+            @Override
+            public WriteResponse apply(Boolean v1) {
+                return ResponseUtils.writeSuccess();
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowTruncate(stream)) {
+            deniedTruncateCounter.inc();
+            throw new RequestDeniedException(stream, "truncate");
+        }
+        super.preExecute();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
new file mode 100644
index 0000000..2e7ffb8
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.RequestDeniedException;
+import org.apache.distributedlog.protocol.util.ProtocolUtils;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+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.AbstractFunction1;
+
+/**
+ * Operation to write a single record to a log stream.
+ */
+public class WriteOp extends AbstractWriteOp implements WriteOpWithPayload {
+
+    private static final Logger logger = LoggerFactory.getLogger(WriteOp.class);
+
+    private final byte[] payload;
+    private final boolean isRecordSet;
+
+    // Stats
+    private final Counter deniedWriteCounter;
+    private final Counter successRecordCounter;
+    private final Counter failureRecordCounter;
+    private final Counter redirectRecordCounter;
+    private final OpStatsLogger latencyStat;
+    private final Counter bytes;
+    private final Counter writeBytes;
+
+    private final byte dlsnVersion;
+    private final AccessControlManager accessControlManager;
+
+    public WriteOp(String stream,
+                   ByteBuffer data,
+                   StatsLogger statsLogger,
+                   StatsLogger perStreamStatsLogger,
+                   StreamPartitionConverter streamPartitionConverter,
+                   ServerConfiguration conf,
+                   byte dlsnVersion,
+                   Long checksum,
+                   boolean isRecordSet,
+                   Feature checksumDisabledFeature,
+                   AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "write"), checksum, checksumDisabledFeature);
+        payload = new byte[data.remaining()];
+        data.get(payload);
+        this.isRecordSet = isRecordSet;
+
+        final Partition partition = streamPartitionConverter.convert(stream);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.successRecordCounter = streamOpStats.recordsCounter("success");
+        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
+        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
+        this.deniedWriteCounter = streamOpStats.requestDeniedCounter("write");
+        this.writeBytes = streamOpStats.scopedRequestCounter("write", "bytes");
+        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "write");
+        this.bytes = streamOpStats.streamRequestCounter(partition, "write", "bytes");
+
+        this.dlsnVersion = dlsnVersion;
+        this.accessControlManager = accessControlManager;
+
+        final long size = getPayloadSize();
+        result().addEventListener(new FutureEventListener<WriteResponse>() {
+            @Override
+            public void onSuccess(WriteResponse response) {
+                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                    bytes.add(size);
+                    writeBytes.add(size);
+                } else {
+                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                }
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+            }
+        });
+    }
+
+    @Override
+    public long getPayloadSize() {
+      return payload.length;
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return ProtocolUtils.writeOpCRC32(stream, payload);
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowWrite(stream)) {
+            deniedWriteCounter.inc();
+            throw new RequestDeniedException(stream, "write");
+        }
+        super.preExecute();
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        if (!stream.equals(writer.getStreamName())) {
+            logger.error("Write: Stream Name Mismatch in the Stream Map {}, {}", stream, writer.getStreamName());
+            return Future.exception(new IllegalStateException("The stream mapping is incorrect, fail the request"));
+        }
+
+        long txnId;
+        Future<DLSN> writeResult;
+        synchronized (txnLock) {
+            txnId = sequencer.nextId();
+            LogRecord record = new LogRecord(txnId, payload);
+            if (isRecordSet) {
+                record.setRecordSet();
+            }
+            writeResult = writer.write(record);
+        }
+        return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
+            @Override
+            public WriteResponse apply(DLSN value) {
+                successRecordCounter.inc();
+                return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
+            }
+        });
+    }
+
+    @Override
+    protected void fail(ResponseHeader header) {
+        if (StatusCode.FOUND == header.getCode()) {
+            redirectRecordCounter.inc();
+        } else {
+            failureRecordCounter.inc();
+        }
+        super.fail(header);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
new file mode 100644
index 0000000..e411b420
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+/**
+ * A write operation with payload.
+ */
+public interface WriteOpWithPayload {
+
+    // Return the payload size in bytes
+    long getPayloadSize();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java
new file mode 100644
index 0000000..fcaee35
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.admin;
+
+import org.apache.distributedlog.exceptions.DLException;
+import com.twitter.util.Future;
+
+/**
+ * Admin operation interface.
+ */
+public interface AdminOp<RespT> {
+
+    /**
+     * Invoked before the stream op is executed.
+     */
+    void preExecute() throws DLException;
+
+    /**
+     * Execute the operation.
+     *
+     * @return the future represents the response of the operation
+     */
+    Future<RespT> execute();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java
new file mode 100644
index 0000000..89a2566
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.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.service.stream.admin;
+
+import static org.apache.distributedlog.service.stream.AbstractStreamOp.requestStat;
+
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to create log stream.
+ */
+public class CreateOp extends StreamAdminOp {
+
+  public CreateOp(String stream,
+                  StatsLogger statsLogger,
+                  StreamManager streamManager,
+                  Long checksum,
+                  Feature checksumEnabledFeature) {
+    super(stream,
+            streamManager,
+            requestStat(statsLogger, "create"),
+            checksum,
+            checksumEnabledFeature);
+  }
+
+  @Override
+  protected Future<WriteResponse> executeOp() {
+    Future<Void> result = streamManager.createStreamAsync(stream);
+    return result.map(new AbstractFunction1<Void, WriteResponse>() {
+      @Override
+      public WriteResponse apply(Void value) {
+        return ResponseUtils.writeSuccess();
+      }
+    });
+  }
+}


[05/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
deleted file mode 100644
index bf7a1ad..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
+++ /dev/null
@@ -1,925 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogManager;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.AlreadyClosedException;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.exceptions.StreamNotReadyException;
-import org.apache.distributedlog.exceptions.StreamUnavailableException;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.io.Abortables;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.service.FatalErrorHandler;
-import org.apache.distributedlog.service.ServerFeatureKeys;
-import org.apache.distributedlog.service.config.ServerConfiguration;
-import org.apache.distributedlog.service.config.StreamConfigProvider;
-import org.apache.distributedlog.service.stream.limiter.StreamRequestLimiter;
-import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.stats.BroadCastStatsLogger;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.TimeSequencer;
-import org.apache.distributedlog.util.Utils;
-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.TimeoutException;
-import com.twitter.util.Timer;
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-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.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.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-/**
- * Implementation of {@link Stream}.
- */
-public class StreamImpl implements Stream {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamImpl.class);
-
-    /**
-     * The status of the stream.
-     *
-     * <p>The status change of the stream should just go in one direction. If a stream hits
-     * any error, the stream should be put in error state. If a stream is in error state,
-     * it should be removed and not reused anymore.
-     */
-    public enum StreamStatus {
-        UNINITIALIZED(-1),
-        INITIALIZING(0),
-        INITIALIZED(1),
-        CLOSING(-4),
-        CLOSED(-5),
-        // if a stream is in error state, it should be abort during closing.
-        ERROR(-6);
-
-        final int code;
-
-        StreamStatus(int code) {
-            this.code = code;
-        }
-
-        int getCode() {
-            return code;
-        }
-
-        public static boolean isUnavailable(StreamStatus status) {
-            return StreamStatus.ERROR == status || StreamStatus.CLOSING == status || StreamStatus.CLOSED == status;
-        }
-    }
-
-    private final String name;
-    private final Partition partition;
-    private DistributedLogManager manager;
-
-    private volatile AsyncLogWriter writer;
-    private volatile StreamStatus status;
-    private volatile String owner;
-    private volatile Throwable lastException;
-    private volatile Queue<StreamOp> pendingOps = new ArrayDeque<StreamOp>();
-
-    private final Promise<Void> closePromise = new Promise<Void>();
-    private final Object txnLock = new Object();
-    private final TimeSequencer sequencer = new TimeSequencer();
-    private final StreamRequestLimiter limiter;
-    private final DynamicDistributedLogConfiguration dynConf;
-    private final DistributedLogConfiguration dlConfig;
-    private final DistributedLogNamespace dlNamespace;
-    private final String clientId;
-    private final OrderedScheduler scheduler;
-    private final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
-    private final Feature featureRateLimitDisabled;
-    private final StreamManager streamManager;
-    private final StreamConfigProvider streamConfigProvider;
-    private final FatalErrorHandler fatalErrorHandler;
-    private final long streamProbationTimeoutMs;
-    private final long serviceTimeoutMs;
-    private final long writerCloseTimeoutMs;
-    private final boolean failFastOnStreamNotReady;
-    private final HashedWheelTimer requestTimer;
-    private final Timer futureTimer;
-
-    // Stats
-    private final StatsLogger streamLogger;
-    private final StatsLogger streamExceptionStatLogger;
-    private final StatsLogger limiterStatLogger;
-    private final Counter serviceTimeout;
-    private final OpStatsLogger streamAcquireStat;
-    private final OpStatsLogger writerCloseStatLogger;
-    private final Counter pendingOpsCounter;
-    private final Counter unexpectedExceptions;
-    private final Counter writerCloseTimeoutCounter;
-    private final StatsLogger exceptionStatLogger;
-    private final ConcurrentHashMap<String, Counter> exceptionCounters =
-        new ConcurrentHashMap<String, Counter>();
-    private final Gauge<Number> streamStatusGauge;
-
-    // Since we may create and discard streams at initialization if there's a race,
-    // must not do any expensive initialization here (particularly any locking or
-    // significant resource allocation etc.).
-    StreamImpl(final String name,
-               final Partition partition,
-               String clientId,
-               StreamManager streamManager,
-               StreamOpStats streamOpStats,
-               ServerConfiguration serverConfig,
-               DistributedLogConfiguration dlConfig,
-               DynamicDistributedLogConfiguration streamConf,
-               FeatureProvider featureProvider,
-               StreamConfigProvider streamConfigProvider,
-               DistributedLogNamespace dlNamespace,
-               OrderedScheduler scheduler,
-               FatalErrorHandler fatalErrorHandler,
-               HashedWheelTimer requestTimer,
-               Timer futureTimer) {
-        this.clientId = clientId;
-        this.dlConfig = dlConfig;
-        this.streamManager = streamManager;
-        this.name = name;
-        this.partition = partition;
-        this.status = StreamStatus.UNINITIALIZED;
-        this.lastException = new IOException("Fail to write record to stream " + name);
-        this.streamConfigProvider = streamConfigProvider;
-        this.dlNamespace = dlNamespace;
-        this.featureRateLimitDisabled = featureProvider.getFeature(
-            ServerFeatureKeys.SERVICE_RATE_LIMIT_DISABLED.name().toLowerCase());
-        this.scheduler = scheduler;
-        this.serviceTimeoutMs = serverConfig.getServiceTimeoutMs();
-        this.streamProbationTimeoutMs = serverConfig.getStreamProbationTimeoutMs();
-        this.writerCloseTimeoutMs = serverConfig.getWriterCloseTimeoutMs();
-        this.failFastOnStreamNotReady = dlConfig.getFailFastOnStreamNotReady();
-        this.fatalErrorHandler = fatalErrorHandler;
-        this.dynConf = streamConf;
-        StatsLogger limiterStatsLogger = BroadCastStatsLogger.two(
-            streamOpStats.baseScope("stream_limiter"),
-            streamOpStats.streamRequestScope(partition, "limiter"));
-        this.limiter = new StreamRequestLimiter(name, dynConf, limiterStatsLogger, featureRateLimitDisabled);
-        this.requestTimer = requestTimer;
-        this.futureTimer = futureTimer;
-
-        // Stats
-        this.streamLogger = streamOpStats.streamRequestStatsLogger(partition);
-        this.limiterStatLogger = streamOpStats.baseScope("request_limiter");
-        this.streamExceptionStatLogger = streamLogger.scope("exceptions");
-        this.serviceTimeout = streamOpStats.baseCounter("serviceTimeout");
-        StatsLogger streamsStatsLogger = streamOpStats.baseScope("streams");
-        this.streamAcquireStat = streamsStatsLogger.getOpStatsLogger("acquire");
-        this.pendingOpsCounter = streamOpStats.baseCounter("pending_ops");
-        this.unexpectedExceptions = streamOpStats.baseCounter("unexpected_exceptions");
-        this.exceptionStatLogger = streamOpStats.requestScope("exceptions");
-        this.writerCloseStatLogger = streamsStatsLogger.getOpStatsLogger("writer_close");
-        this.writerCloseTimeoutCounter = streamsStatsLogger.getCounter("writer_close_timeouts");
-        // Gauges
-        this.streamStatusGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return StreamStatus.UNINITIALIZED.getCode();
-            }
-            @Override
-            public Number getSample() {
-                return status.getCode();
-            }
-        };
-    }
-
-    @Override
-    public String getOwner() {
-        return owner;
-    }
-
-    @Override
-    public String getStreamName() {
-        return name;
-    }
-
-    @Override
-    public DynamicDistributedLogConfiguration getStreamConfiguration() {
-        return dynConf;
-    }
-
-    @Override
-    public Partition getPartition() {
-        return partition;
-    }
-
-    private DistributedLogManager openLog(String name) throws IOException {
-        Optional<DistributedLogConfiguration> dlConf = Optional.<DistributedLogConfiguration>absent();
-        Optional<DynamicDistributedLogConfiguration> dynDlConf = Optional.of(dynConf);
-        Optional<StatsLogger> perStreamStatsLogger = Optional.of(streamLogger);
-        return dlNamespace.openLog(name, dlConf, dynDlConf, perStreamStatsLogger);
-    }
-
-    // Expensive initialization, only called once per stream.
-    @Override
-    public void initialize() throws IOException {
-        manager = openLog(name);
-
-        // Better to avoid registering the gauge multiple times, so do this in init
-        // which only gets called once.
-        streamLogger.registerGauge("stream_status", this.streamStatusGauge);
-
-        // Signal initialization is complete, should be last in this method.
-        status = StreamStatus.INITIALIZING;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("Stream:%s, %s, %s Status:%s", name, manager, writer, status);
-    }
-
-    @Override
-    public void start() {
-        // acquire the stream
-        acquireStream().addEventListener(new FutureEventListener<Boolean>() {
-                @Override
-                public void onSuccess(Boolean success) {
-                    if (!success) {
-                        // failed to acquire the stream. set the stream in error status and close it.
-                        setStreamInErrorStatus();
-                        requestClose("Failed to acquire the ownership");
-                    }
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    // unhandled exceptions
-                    logger.error("Stream {} threw unhandled exception : ", name, cause);
-                    // failed to acquire the stream. set the stream in error status and close it.
-                    setStreamInErrorStatus();
-                    requestClose("Unhandled exception");
-                }
-            });
-    }
-
-    //
-    // Stats Operations
-    //
-
-    void countException(Throwable t, StatsLogger streamExceptionLogger) {
-        String exceptionName = null == t ? "null" : t.getClass().getName();
-        Counter counter = exceptionCounters.get(exceptionName);
-        if (null == counter) {
-            counter = exceptionStatLogger.getCounter(exceptionName);
-            Counter oldCounter = exceptionCounters.putIfAbsent(exceptionName, counter);
-            if (null != oldCounter) {
-                counter = oldCounter;
-            }
-        }
-        counter.inc();
-        streamExceptionLogger.getCounter(exceptionName).inc();
-    }
-
-    boolean isCriticalException(Throwable cause) {
-        return !(cause instanceof OwnershipAcquireFailedException);
-    }
-
-    //
-    // Service Timeout:
-    // - schedule a timeout function to handle operation timeouts: {@link #handleServiceTimeout(String)}
-    // - if the operation is completed within timeout period, cancel the timeout.
-    //
-
-    void scheduleTimeout(final StreamOp op) {
-        final Timeout timeout = requestTimer.newTimeout(new TimerTask() {
-            @Override
-            public void run(Timeout timeout) throws Exception {
-                if (!timeout.isCancelled()) {
-                    serviceTimeout.inc();
-                    handleServiceTimeout("Operation " + op.getClass().getName() + " timeout");
-                }
-            }
-        }, serviceTimeoutMs, TimeUnit.MILLISECONDS);
-        op.responseHeader().ensure(new Function0<BoxedUnit>() {
-            @Override
-            public BoxedUnit apply() {
-                timeout.cancel();
-                return null;
-            }
-        });
-    }
-
-    /**
-     * Close the stream and schedule cache eviction at some point in the future.
-     * We delay this as a way to place the stream in a probationary state--cached
-     * in the proxy but unusable.
-     * This mechanism helps the cluster adapt to situations where a proxy has
-     * persistent connectivity/availability issues, because it keeps an affected
-     * stream off the proxy for a period of time, hopefully long enough for the
-     * issues to be resolved, or for whoop to kick in and kill the shard.
-     */
-    void handleServiceTimeout(String reason) {
-        synchronized (this) {
-            if (StreamStatus.isUnavailable(status)) {
-                return;
-            }
-            // Mark stream in error state
-            setStreamInErrorStatus();
-        }
-
-        // Async close request, and schedule eviction when its done.
-        Future<Void> closeFuture = requestClose(reason, false /* dont remove */);
-        closeFuture.onSuccess(new AbstractFunction1<Void, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Void result) {
-                streamManager.scheduleRemoval(StreamImpl.this, streamProbationTimeoutMs);
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    //
-    // Submit the operation to the stream.
-    //
-
-    /**
-     * Execute the StreamOp. If reacquire is needed, this may initiate reacquire and queue the op for
-     * execution once complete.
-     *
-     * @param op
-     *          stream operation to execute.
-     */
-    @Override
-    public void submit(StreamOp op) {
-        try {
-            limiter.apply(op);
-        } catch (OverCapacityException ex) {
-            op.fail(ex);
-            return;
-        }
-
-        // Timeout stream op if requested.
-        if (serviceTimeoutMs > 0) {
-            scheduleTimeout(op);
-        }
-
-        boolean completeOpNow = false;
-        boolean success = true;
-        if (StreamStatus.isUnavailable(status)) {
-            // Stream is closed, fail the op immediately
-            op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
-            return;
-        } else if (StreamStatus.INITIALIZED == status && writer != null) {
-            completeOpNow = true;
-            success = true;
-        } else {
-            synchronized (this) {
-                if (StreamStatus.isUnavailable(status)) {
-                    // Stream is closed, fail the op immediately
-                    op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
-                    return;
-                } else if (StreamStatus.INITIALIZED == status) {
-                    completeOpNow = true;
-                    success = true;
-                } else if (failFastOnStreamNotReady) {
-                    op.fail(new StreamNotReadyException("Stream " + name + " is not ready; status = " + status));
-                    return;
-                } else { // the stream is still initializing
-                    pendingOps.add(op);
-                    pendingOpsCounter.inc();
-                    if (1 == pendingOps.size()) {
-                        if (op instanceof HeartbeatOp) {
-                            ((HeartbeatOp) op).setWriteControlRecord(true);
-                        }
-                    }
-                }
-            }
-        }
-        if (completeOpNow) {
-            executeOp(op, success);
-        }
-    }
-
-    //
-    // Execute operations and handle exceptions on operations
-    //
-
-    /**
-     * Execute the <i>op</i> immediately.
-     *
-     * @param op
-     *          stream operation to execute.
-     * @param success
-     *          whether the operation is success or not.
-     */
-    void executeOp(final StreamOp op, boolean success) {
-        final AsyncLogWriter writer;
-        final Throwable lastException;
-        synchronized (this) {
-            writer = this.writer;
-            lastException = this.lastException;
-        }
-        if (null != writer && success) {
-            op.execute(writer, sequencer, txnLock)
-                    .addEventListener(new FutureEventListener<Void>() {
-                @Override
-                public void onSuccess(Void value) {
-                    // nop
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    boolean countAsException = true;
-                    if (cause instanceof DLException) {
-                        final DLException dle = (DLException) cause;
-                        switch (dle.getCode()) {
-                        case FOUND:
-                            assert(cause instanceof OwnershipAcquireFailedException);
-                            countAsException = false;
-                            handleExceptionOnStreamOp(op, cause);
-                            break;
-                        case ALREADY_CLOSED:
-                            assert(cause instanceof AlreadyClosedException);
-                            op.fail(cause);
-                            handleAlreadyClosedException((AlreadyClosedException) cause);
-                            break;
-                        // exceptions that mostly from client (e.g. too large record)
-                        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 TOO_LARGE_RECORD:
-                        case STREAM_NOT_READY:
-                        case OVER_CAPACITY:
-                            op.fail(cause);
-                            break;
-                        // the DL writer hits exception, simple set the stream to error status
-                        // and fail the request
-                        default:
-                            handleExceptionOnStreamOp(op, cause);
-                            break;
-                        }
-                    } else {
-                        handleExceptionOnStreamOp(op, cause);
-                    }
-                    if (countAsException) {
-                        countException(cause, streamExceptionStatLogger);
-                    }
-                }
-            });
-        } else {
-            if (null != lastException) {
-                op.fail(lastException);
-            } else {
-                op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
-            }
-        }
-    }
-
-    /**
-     * Handle exception when executing <i>op</i>.
-     *
-     * @param op
-     *          stream operation executing
-     * @param cause
-     *          exception received when executing <i>op</i>
-     */
-    private void handleExceptionOnStreamOp(StreamOp op, final Throwable cause) {
-        AsyncLogWriter oldWriter = null;
-        boolean statusChanged = false;
-        synchronized (this) {
-            if (StreamStatus.INITIALIZED == status) {
-                oldWriter = setStreamStatus(StreamStatus.ERROR, StreamStatus.INITIALIZED, null, cause);
-                statusChanged = true;
-            }
-        }
-        if (statusChanged) {
-            Abortables.asyncAbort(oldWriter, false);
-            if (isCriticalException(cause)) {
-                logger.error("Failed to write data into stream {} : ", name, cause);
-            } else {
-                logger.warn("Failed to write data into stream {} : {}", name, cause.getMessage());
-            }
-            requestClose("Failed to write data into stream " + name + " : " + cause.getMessage());
-        }
-        op.fail(cause);
-    }
-
-    /**
-     * Handling already closed exception.
-     */
-    private void handleAlreadyClosedException(AlreadyClosedException ace) {
-        unexpectedExceptions.inc();
-        logger.error("Encountered unexpected exception when writing data into stream {} : ", name, ace);
-        fatalErrorHandler.notifyFatalError();
-    }
-
-    //
-    // Acquire streams
-    //
-
-    Future<Boolean> acquireStream() {
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        final Promise<Boolean> acquirePromise = new Promise<Boolean>();
-        manager.openAsyncLogWriter().addEventListener(
-            FutureUtils.OrderedFutureEventListener.of(new FutureEventListener<AsyncLogWriter>() {
-
-            @Override
-            public void onSuccess(AsyncLogWriter w) {
-                onAcquireStreamSuccess(w, stopwatch, acquirePromise);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                onAcquireStreamFailure(cause, stopwatch, acquirePromise);
-            }
-
-        }, scheduler, getStreamName()));
-        return acquirePromise;
-    }
-
-    private void onAcquireStreamSuccess(AsyncLogWriter w,
-                                        Stopwatch stopwatch,
-                                        Promise<Boolean> acquirePromise) {
-        synchronized (txnLock) {
-            sequencer.setLastId(w.getLastTxId());
-        }
-        AsyncLogWriter oldWriter;
-        Queue<StreamOp> oldPendingOps;
-        boolean success;
-        synchronized (StreamImpl.this) {
-            oldWriter = setStreamStatus(StreamStatus.INITIALIZED,
-                    StreamStatus.INITIALIZING, w, null);
-            oldPendingOps = pendingOps;
-            pendingOps = new ArrayDeque<StreamOp>();
-            success = true;
-        }
-        // check if the stream is allowed to be acquired
-        if (!streamManager.allowAcquire(StreamImpl.this)) {
-            if (null != oldWriter) {
-                Abortables.asyncAbort(oldWriter, true);
-            }
-            int maxAcquiredPartitions = dynConf.getMaxAcquiredPartitionsPerProxy();
-            StreamUnavailableException sue = new StreamUnavailableException("Stream " + partition.getStream()
-                    + " is not allowed to acquire more than " + maxAcquiredPartitions + " partitions");
-            countException(sue, exceptionStatLogger);
-            logger.error("Failed to acquire stream {} because it is unavailable : {}",
-                    name, sue.getMessage());
-            synchronized (this) {
-                oldWriter = setStreamStatus(StreamStatus.ERROR,
-                        StreamStatus.INITIALIZED, null, sue);
-                // we don't switch the pending ops since they are already switched
-                // when setting the status to initialized
-                success = false;
-            }
-        }
-        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
-    }
-
-    private void onAcquireStreamFailure(Throwable cause,
-                                        Stopwatch stopwatch,
-                                        Promise<Boolean> acquirePromise) {
-        AsyncLogWriter oldWriter;
-        Queue<StreamOp> oldPendingOps;
-        boolean success;
-        if (cause instanceof AlreadyClosedException) {
-            countException(cause, streamExceptionStatLogger);
-            handleAlreadyClosedException((AlreadyClosedException) cause);
-            return;
-        } else {
-            if (isCriticalException(cause)) {
-                countException(cause, streamExceptionStatLogger);
-                logger.error("Failed to acquire stream {} : ", name, cause);
-            } else {
-                logger.warn("Failed to acquire stream {} : {}", name, cause.getMessage());
-            }
-            synchronized (StreamImpl.this) {
-                oldWriter = setStreamStatus(StreamStatus.ERROR,
-                        StreamStatus.INITIALIZING, null, cause);
-                oldPendingOps = pendingOps;
-                pendingOps = new ArrayDeque<StreamOp>();
-                success = false;
-            }
-        }
-        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
-    }
-
-    /**
-     * Process the pending request after acquired stream.
-     *
-     * @param success whether the acquisition succeed or not
-     * @param oldWriter the old writer to abort
-     * @param oldPendingOps the old pending ops to execute
-     * @param stopwatch stopwatch to measure the time spent on acquisition
-     * @param acquirePromise the promise to complete the acquire operation
-     */
-    void processPendingRequestsAfterAcquire(boolean success,
-                                            AsyncLogWriter oldWriter,
-                                            Queue<StreamOp> oldPendingOps,
-                                            Stopwatch stopwatch,
-                                            Promise<Boolean> acquirePromise) {
-        if (success) {
-            streamAcquireStat.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-        } else {
-            streamAcquireStat.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-        }
-        for (StreamOp op : oldPendingOps) {
-            executeOp(op, success);
-            pendingOpsCounter.dec();
-        }
-        Abortables.asyncAbort(oldWriter, true);
-        FutureUtils.setValue(acquirePromise, success);
-    }
-
-    //
-    // Stream Status Changes
-    //
-
-    synchronized void setStreamInErrorStatus() {
-        if (StreamStatus.CLOSING == status || StreamStatus.CLOSED == status) {
-            return;
-        }
-        this.status = StreamStatus.ERROR;
-    }
-
-    /**
-     * Update the stream status. The changes are only applied when there isn't status changed.
-     *
-     * @param newStatus
-     *          new status
-     * @param oldStatus
-     *          old status
-     * @param writer
-     *          new log writer
-     * @param t
-     *          new exception
-     * @return old writer if it exists
-     */
-    synchronized AsyncLogWriter setStreamStatus(StreamStatus newStatus,
-                                                StreamStatus oldStatus,
-                                                AsyncLogWriter writer,
-                                                Throwable t) {
-        if (oldStatus != this.status) {
-            logger.info("Stream {} status already changed from {} -> {} when trying to change it to {}",
-                    new Object[] { name, oldStatus, this.status, newStatus });
-            return null;
-        }
-
-        String owner = null;
-        if (t instanceof OwnershipAcquireFailedException) {
-            owner = ((OwnershipAcquireFailedException) t).getCurrentOwner();
-        }
-
-        AsyncLogWriter oldWriter = this.writer;
-        this.writer = writer;
-        if (null != owner && owner.equals(clientId)) {
-            unexpectedExceptions.inc();
-            logger.error("I am waiting myself {} to release lock on stream {}, so have to shut myself down :",
-                         new Object[] { owner, name, t });
-            // I lost the ownership but left a lock over zookeeper
-            // I should not ask client to redirect to myself again as I can't handle it :(
-            // shutdown myself
-            fatalErrorHandler.notifyFatalError();
-            this.owner = null;
-        } else {
-            this.owner = owner;
-        }
-        this.lastException = t;
-        this.status = newStatus;
-        if (StreamStatus.INITIALIZED == newStatus) {
-            streamManager.notifyAcquired(this);
-            logger.info("Inserted acquired stream {} -> writer {}", name, this);
-        } else {
-            streamManager.notifyReleased(this);
-            logger.info("Removed acquired stream {} -> writer {}", name, this);
-        }
-        return oldWriter;
-    }
-
-    //
-    // Stream Close Functions
-    //
-
-    void close(DistributedLogManager dlm) {
-        if (null != dlm) {
-            try {
-                dlm.close();
-            } catch (IOException ioe) {
-                logger.warn("Failed to close dlm for {} : ", name, ioe);
-            }
-        }
-    }
-
-    @Override
-    public Future<Void> requestClose(String reason) {
-        return requestClose(reason, true);
-    }
-
-    Future<Void> requestClose(String reason, boolean uncache) {
-        final boolean abort;
-        closeLock.writeLock().lock();
-        try {
-            if (StreamStatus.CLOSING == status
-                || StreamStatus.CLOSED == status) {
-                return closePromise;
-            }
-            logger.info("Request to close stream {} : {}", getStreamName(), reason);
-            // if the stream isn't closed from INITIALIZED state, we abort the stream instead of closing it.
-            abort = StreamStatus.INITIALIZED != status;
-            status = StreamStatus.CLOSING;
-            streamManager.notifyReleased(this);
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-        // we will fail the requests that are coming in between closing and closed only
-        // after the async writer is closed. so we could clear up the lock before redirect
-        // them.
-        close(abort, uncache);
-        return closePromise;
-    }
-
-    @Override
-    public void delete() throws IOException {
-        if (null != writer) {
-            Utils.close(writer);
-            synchronized (this) {
-                writer = null;
-                lastException = new StreamUnavailableException("Stream was deleted");
-            }
-        }
-        if (null == manager) {
-            throw new UnexpectedException("No stream " + name + " to delete");
-        }
-        manager.delete();
-    }
-
-    /**
-     * Post action executed after closing.
-     */
-    private void postClose(boolean uncache) {
-        closeManagerAndErrorOutPendingRequests();
-        unregisterGauge();
-        if (uncache) {
-            if (null != owner) {
-                long probationTimeoutMs = 2 * dlConfig.getZKSessionTimeoutMilliseconds() / 3;
-                streamManager.scheduleRemoval(this, probationTimeoutMs);
-            } else {
-                streamManager.notifyRemoved(this);
-                logger.info("Removed cached stream {}.", getStreamName());
-            }
-        }
-        FutureUtils.setValue(closePromise, null);
-    }
-
-    /**
-     * Shouldn't call close directly. The callers should call #requestClose instead
-     *
-     * @param shouldAbort shall we abort the stream instead of closing
-     */
-    private Future<Void> close(boolean shouldAbort, final boolean uncache) {
-        boolean abort;
-        closeLock.writeLock().lock();
-        try {
-            if (StreamStatus.CLOSED == status) {
-                return closePromise;
-            }
-            abort = shouldAbort || (StreamStatus.INITIALIZED != status && StreamStatus.CLOSING != status);
-            status = StreamStatus.CLOSED;
-            streamManager.notifyReleased(this);
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-        logger.info("Closing stream {} ...", name);
-        // Close the writers to release the locks before failing the requests
-        Future<Void> closeWriterFuture;
-        if (abort) {
-            closeWriterFuture = Abortables.asyncAbort(writer, true);
-        } else {
-            closeWriterFuture = Utils.asyncClose(writer, true);
-        }
-        // close the manager and error out pending requests after close writer
-        Duration closeWaitDuration;
-        if (writerCloseTimeoutMs <= 0) {
-            closeWaitDuration = Duration.Top();
-        } else {
-            closeWaitDuration = Duration.fromMilliseconds(writerCloseTimeoutMs);
-        }
-
-        FutureUtils.stats(
-                closeWriterFuture,
-                writerCloseStatLogger,
-                Stopwatch.createStarted()
-        ).masked().within(futureTimer, closeWaitDuration)
-                .addEventListener(FutureUtils.OrderedFutureEventListener.of(
-                new FutureEventListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        postClose(uncache);
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        if (cause instanceof TimeoutException) {
-                            writerCloseTimeoutCounter.inc();
-                        }
-                        postClose(uncache);
-                    }
-                }, scheduler, name));
-        return closePromise;
-    }
-
-    private void closeManagerAndErrorOutPendingRequests() {
-        close(manager);
-        // Failed the pending requests.
-        Queue<StreamOp> oldPendingOps;
-        synchronized (this) {
-            oldPendingOps = pendingOps;
-            pendingOps = new ArrayDeque<StreamOp>();
-        }
-        StreamUnavailableException closingException =
-                new StreamUnavailableException("Stream " + name + " is closed.");
-        for (StreamOp op : oldPendingOps) {
-            op.fail(closingException);
-            pendingOpsCounter.dec();
-        }
-        limiter.close();
-        logger.info("Closed stream {}.", name);
-    }
-
-    /**
-     * clean up the gauge to help GC.
-     */
-    private void unregisterGauge(){
-        streamLogger.unregisterGauge("stream_status", this.streamStatusGauge);
-    }
-
-    // Test-only apis
-
-    @VisibleForTesting
-    public int numPendingOps() {
-        Queue<StreamOp> queue = pendingOps;
-        return null == queue ? 0 : queue.size();
-    }
-
-    @VisibleForTesting
-    public StreamStatus getStatus() {
-        return status;
-    }
-
-    @VisibleForTesting
-    public void setStatus(StreamStatus status) {
-        this.status = status;
-    }
-
-    @VisibleForTesting
-    public AsyncLogWriter getWriter() {
-        return writer;
-    }
-
-    @VisibleForTesting
-    public DistributedLogManager getManager() {
-        return manager;
-    }
-
-    @VisibleForTesting
-    public Throwable getLastException() {
-        return lastException;
-    }
-
-    @VisibleForTesting
-    public Future<Void> getCloseFuture() {
-        return closePromise;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
deleted file mode 100644
index d86c538..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import com.google.common.base.Optional;
-import com.twitter.util.Future;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Manage lifecycle of streams.
- *
- * <p>StreamManager is responsible for creating, destroying, and keeping track of Stream objects.
- *
- * <p>Stream objects, which are managed by StreamManager and created by StreamFactory, are essentially the
- * per stream request handlers, responsible fo dispatching ex. write requests to an underlying AsyncLogWriter,
- * managing stream lock, interpreting exceptions, error conditions, and etc.
- */
-public interface StreamManager {
-
-    /**
-     * Get a cached stream, returning null if it doesnt exist.
-     * @param stream name
-     * @return the cached stream
-     */
-    Stream getStream(String stream);
-
-    /**
-     * Get a cached stream and create a new one if it doesnt exist.
-     * @param streamName stream name
-     * @param start whether to start the stream after it is created.
-     * @return future satisfied once close complete
-     */
-    Stream getOrCreateStream(String streamName, boolean start) throws IOException;
-
-    /**
-     * Asynchronously create a new stream.
-     * @param stream
-     * @return Future satisfied once the stream is created
-     */
-    Future<Void> createStreamAsync(String stream);
-
-    /**
-     * Is acquiring stream allowed?
-     *
-     * @param stream
-     *          stream instance
-     * @return true if it is allowed to acquire this stream, otherwise false.
-     */
-    boolean allowAcquire(Stream stream);
-
-    /**
-     * Notify the manager that a stream was acquired.
-     * @param stream being acquired
-     */
-    void notifyAcquired(Stream stream);
-
-    /**
-     * Notify the manager that a stream was released.
-     * @param stream being released
-     */
-    void notifyReleased(Stream stream);
-
-    /**
-     * Notify the manager that a stream was completely removed.
-     * @param stream being uncached
-     * @return whether the stream existed or not
-     */
-    boolean notifyRemoved(Stream stream);
-
-    /**
-     * Asynchronous delete method.
-     * @param streamName stream name
-     * @return future satisfied once delete complete
-     */
-    Future<Void> deleteAndRemoveAsync(String streamName);
-
-    /**
-     * Asynchronous close and uncache method.
-     * @param streamName stream name
-     * @return future satisfied once close and uncache complete
-     */
-    Future<Void> closeAndRemoveAsync(String streamName);
-
-    /**
-     * Close and uncache after delayMs.
-     * @param stream to remove
-     */
-    void scheduleRemoval(Stream stream, long delayMs);
-
-    /**
-     * Close all stream.
-     * @return future satisfied all streams closed
-     */
-    Future<List<Void>> closeStreams();
-
-    /**
-     * Return map with stream ownership info.
-     * @param regex for filtering streams
-     * @return map containing ownership info
-     */
-    Map<String, String> getStreamOwnershipMap(Optional<String> regex);
-
-    /**
-     * Number of acquired streams.
-     * @return number of acquired streams
-     */
-    int numAcquired();
-
-    /**
-     * Number of cached streams.
-     * @return number of cached streams
-     */
-    int numCached();
-
-    /**
-     * Is the stream denoted by streamName in the acquired state.
-     * @return true if the stream is in the acquired state
-     */
-    boolean isAcquired(String streamName);
-
-    /**
-     * Close manager and disallow further activity.
-     */
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
deleted file mode 100644
index 5d54738..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
+++ /dev/null
@@ -1,413 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.ServiceUnavailableException;
-import org.apache.distributedlog.exceptions.StreamUnavailableException;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.service.config.StreamConfigProvider;
-import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.service.streamset.PartitionMap;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.distributedlog.util.ConfUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * StreamManagerImpl is the default implementation responsible for creating, destroying, and keeping track
- * of Streams.
- *
- * <p>StreamFactory, supplied to StreamManagerImpl in the constructor below, is reposible simply for creating
- * a stream object in isolation from the rest of the system. We pass a StreamFactory in instead of simply
- * creating StreamImpl's ourselves in order to inject dependencies without bloating the StreamManagerImpl
- * constructor.
- */
-public class StreamManagerImpl implements StreamManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamManagerImpl.class);
-
-    private final ConcurrentHashMap<String, Stream> streams =
-        new ConcurrentHashMap<String, Stream>();
-    private final AtomicInteger numCached = new AtomicInteger(0);
-
-    private final ConcurrentHashMap<String, Stream> acquiredStreams =
-        new ConcurrentHashMap<String, Stream>();
-    private final AtomicInteger numAcquired = new AtomicInteger(0);
-
-    //
-    // Partitions
-    //
-    private final StreamPartitionConverter partitionConverter;
-    private final PartitionMap cachedPartitions = new PartitionMap();
-    private final PartitionMap acquiredPartitions = new PartitionMap();
-
-    final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
-    private final ScheduledExecutorService executorService;
-    private final DistributedLogConfiguration dlConfig;
-    private final StreamConfigProvider streamConfigProvider;
-    private final String clientId;
-    private boolean closed = false;
-    private final StreamFactory streamFactory;
-    private final DistributedLogNamespace dlNamespace;
-
-    public StreamManagerImpl(String clientId,
-                             DistributedLogConfiguration dlConfig,
-                             ScheduledExecutorService executorService,
-                             StreamFactory streamFactory,
-                             StreamPartitionConverter partitionConverter,
-                             StreamConfigProvider streamConfigProvider,
-                             DistributedLogNamespace dlNamespace) {
-        this.clientId = clientId;
-        this.executorService = executorService;
-        this.streamFactory = streamFactory;
-        this.partitionConverter = partitionConverter;
-        this.dlConfig = dlConfig;
-        this.streamConfigProvider = streamConfigProvider;
-        this.dlNamespace = dlNamespace;
-    }
-
-    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
-        Optional<DynamicDistributedLogConfiguration> dynDlConf =
-                streamConfigProvider.getDynamicStreamConfig(streamName);
-        if (dynDlConf.isPresent()) {
-            return dynDlConf.get();
-        } else {
-            return ConfUtils.getConstDynConf(dlConfig);
-        }
-    }
-
-    @Override
-    public boolean allowAcquire(Stream stream) {
-        return acquiredPartitions.addPartition(
-                stream.getPartition(),
-                stream.getStreamConfiguration().getMaxAcquiredPartitionsPerProxy());
-    }
-
-    /**
-     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
-     * try to acquire the same read-write lock).
-     */
-    @Override
-    public Future<Void> deleteAndRemoveAsync(final String stream) {
-        final Promise<Void> result = new Promise<Void>();
-        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
-            @Override
-            public void run() {
-                result.become(doDeleteAndRemoveAsync(stream));
-            }
-        }, 0);
-        if (null == scheduleFuture) {
-            return Future.exception(
-                new ServiceUnavailableException("Couldn't schedule a delete task."));
-        }
-        return result;
-    }
-
-    /**
-     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
-     * try to acquire the same read-write lock).
-     */
-    @Override
-    public Future<Void> closeAndRemoveAsync(final String streamName) {
-        final Promise<Void> releasePromise = new Promise<Void>();
-        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
-            @Override
-            public void run() {
-                releasePromise.become(doCloseAndRemoveAsync(streamName));
-            }
-        }, 0);
-        if (null == scheduleFuture) {
-            return Future.exception(
-                new ServiceUnavailableException("Couldn't schedule a release task."));
-        }
-        return releasePromise;
-    }
-
-    @Override
-    public Future<Void> createStreamAsync(final String stream) {
-        final Promise<Void> createPromise = new Promise<Void>();
-        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    dlNamespace.createLog(stream);
-                    createPromise.setValue(null);
-                } catch (Exception e) {
-                    createPromise.setException(e);
-                }
-            }
-        }, 0);
-        if (null == scheduleFuture) {
-            return Future.exception(
-                new ServiceUnavailableException("Couldn't schedule a create task."));
-        }
-        return createPromise;
-    }
-
-    @Override
-    public void notifyReleased(Stream stream) {
-        acquiredPartitions.removePartition(stream.getPartition());
-        if (acquiredStreams.remove(stream.getStreamName(), stream)) {
-            numAcquired.getAndDecrement();
-        }
-    }
-
-    @Override
-    public void notifyAcquired(Stream stream) {
-        if (null == acquiredStreams.put(stream.getStreamName(), stream)) {
-            numAcquired.getAndIncrement();
-        }
-    }
-
-    @Override
-    public boolean notifyRemoved(Stream stream) {
-        cachedPartitions.removePartition(stream.getPartition());
-        if (streams.remove(stream.getStreamName(), stream)) {
-            numCached.getAndDecrement();
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public Map<String, String> getStreamOwnershipMap(Optional<String> regex) {
-        Map<String, String> ownershipMap = new HashMap<String, String>();
-        for (Map.Entry<String, Stream> entry : acquiredStreams.entrySet()) {
-            String name = entry.getKey();
-            if (regex.isPresent() && !name.matches(regex.get())) {
-                continue;
-            }
-            Stream stream = entry.getValue();
-            if (null == stream) {
-                continue;
-            }
-            String owner = stream.getOwner();
-            if (null == owner) {
-                ownershipMap.put(name, clientId);
-            }
-        }
-        return ownershipMap;
-    }
-
-    @Override
-    public Stream getStream(String stream) {
-        return streams.get(stream);
-    }
-
-    @Override
-    public Stream getOrCreateStream(String streamName, boolean start) throws IOException {
-        Stream stream = streams.get(streamName);
-        if (null == stream) {
-            closeLock.readLock().lock();
-            try {
-                if (closed) {
-                    return null;
-                }
-                DynamicDistributedLogConfiguration dynConf = getDynConf(streamName);
-                int maxCachedPartitions = dynConf.getMaxCachedPartitionsPerProxy();
-
-                // get partition from the stream name
-                Partition partition = partitionConverter.convert(streamName);
-
-                // add partition to cached map
-                if (!cachedPartitions.addPartition(partition, maxCachedPartitions)) {
-                    throw new StreamUnavailableException("Stream " + streamName
-                            + " is not allowed to cache more than " + maxCachedPartitions + " partitions");
-                }
-
-                stream = newStream(streamName, dynConf);
-                Stream oldWriter = streams.putIfAbsent(streamName, stream);
-                if (null != oldWriter) {
-                    stream = oldWriter;
-                } else {
-                    numCached.getAndIncrement();
-                    logger.info("Inserted mapping stream name {} -> stream {}", streamName, stream);
-                    stream.initialize();
-                    if (start) {
-                        stream.start();
-                    }
-                }
-            } finally {
-                closeLock.readLock().unlock();
-            }
-        }
-        return stream;
-    }
-
-    @Override
-    public Future<List<Void>> closeStreams() {
-        int numAcquired = acquiredStreams.size();
-        int numCached = streams.size();
-        logger.info("Closing all acquired streams : acquired = {}, cached = {}.",
-            numAcquired, numCached);
-        Set<Stream> streamsToClose = new HashSet<Stream>();
-        streamsToClose.addAll(streams.values());
-        return closeStreams(streamsToClose, Optional.<RateLimiter>absent());
-    }
-
-    @Override
-    public void scheduleRemoval(final Stream stream, long delayMs) {
-        if (delayMs > 0) {
-            logger.info("Scheduling removal of stream {} from cache after {} sec.",
-                    stream.getStreamName(), delayMs);
-        }
-        schedule(new Runnable() {
-            @Override
-            public void run() {
-                if (notifyRemoved(stream)) {
-                    logger.info("Removed cached stream {} after probation.", stream.getStreamName());
-                } else {
-                    logger.info("Cached stream {} already removed.", stream.getStreamName());
-                }
-            }
-        }, delayMs);
-    }
-
-    @Override
-    public int numAcquired() {
-        return numAcquired.get();
-    }
-
-    @Override
-    public int numCached() {
-        return numCached.get();
-    }
-
-    @Override
-    public boolean isAcquired(String streamName) {
-        return acquiredStreams.containsKey(streamName);
-    }
-
-    @Override
-    public void close() {
-        closeLock.writeLock().lock();
-        try {
-            if (closed) {
-                return;
-            }
-            closed = true;
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-    }
-
-    private Future<List<Void>> closeStreams(Set<Stream> streamsToClose, Optional<RateLimiter> rateLimiter) {
-        if (streamsToClose.isEmpty()) {
-            logger.info("No streams to close.");
-            List<Void> emptyList = new ArrayList<Void>();
-            return Future.value(emptyList);
-        }
-        List<Future<Void>> futures = new ArrayList<Future<Void>>(streamsToClose.size());
-        for (Stream stream : streamsToClose) {
-            if (rateLimiter.isPresent()) {
-                rateLimiter.get().acquire();
-            }
-            futures.add(stream.requestClose("Close Streams"));
-        }
-        return Future.collect(futures);
-    }
-
-    private Stream newStream(String name, DynamicDistributedLogConfiguration streamConf) {
-        return streamFactory.create(name, streamConf, this);
-    }
-
-    public Future<Void> doCloseAndRemoveAsync(final String streamName) {
-        Stream stream = streams.get(streamName);
-        if (null == stream) {
-            logger.info("No stream {} to release.", streamName);
-            return Future.value(null);
-        } else {
-            return stream.requestClose("release ownership");
-        }
-    }
-
-    /**
-     * Dont schedule if we're closed - closeLock is acquired to close, so if we acquire the
-     * lock and discover we're not closed, we won't schedule.
-     */
-    private java.util.concurrent.Future<?> schedule(Runnable runnable, long delayMs) {
-        closeLock.readLock().lock();
-        try {
-            if (closed) {
-                return null;
-            } else if (delayMs > 0) {
-                return executorService.schedule(runnable, delayMs, TimeUnit.MILLISECONDS);
-            } else {
-                return executorService.submit(runnable);
-            }
-        } catch (RejectedExecutionException ree) {
-            logger.error("Failed to schedule task {} in {} ms : ",
-                    new Object[] { runnable, delayMs, ree });
-            return null;
-        } finally {
-            closeLock.readLock().unlock();
-        }
-    }
-
-    private Future<Void> doDeleteAndRemoveAsync(final String streamName) {
-        Stream stream = streams.get(streamName);
-        if (null == stream) {
-            logger.warn("No stream {} to delete.", streamName);
-            return Future.exception(new UnexpectedException("No stream " + streamName + " to delete."));
-        } else {
-            Future<Void> result;
-            logger.info("Deleting stream {}, {}", streamName, stream);
-            try {
-                stream.delete();
-                result = stream.requestClose("Stream Deleted");
-            } catch (IOException e) {
-                logger.error("Failed on removing stream {} : ", streamName, e);
-                result = Future.exception(e);
-            }
-            return result;
-        }
-    }
-
-    @VisibleForTesting
-    public ConcurrentHashMap<String, Stream> getCachedStreams() {
-        return streams;
-    }
-
-    @VisibleForTesting
-    public ConcurrentHashMap<String, Stream> getAcquiredStreams() {
-        return acquiredStreams;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
deleted file mode 100644
index d0b8de4..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-
-/**
- * An operation applied to a stream.
- */
-public interface StreamOp {
-    /**
-     * Execute a stream op with the supplied writer.
-     *
-     * @param writer active writer for applying the change
-     * @param sequencer sequencer used for generating transaction id for stream operations
-     * @param txnLock transaction lock to guarantee ordering of transaction id
-     * @return a future satisfied when the operation completes execution
-     */
-    Future<Void> execute(AsyncLogWriter writer,
-                         Sequencer sequencer,
-                         Object txnLock);
-
-    /**
-     * Invoked before the stream op is executed.
-     */
-    void preExecute() throws DLException;
-
-    /**
-     * Return the response header (containing the status code etc.).
-     *
-     * @return A future containing the response header or the exception
-     *      encountered by the op if it failed.
-     */
-    Future<ResponseHeader> responseHeader();
-
-    /**
-     * Abort the operation with the givem exception.
-     */
-    void fail(Throwable t);
-
-    /**
-     * Return the stream name.
-     */
-    String streamName();
-
-    /**
-     * Stopwatch gives the start time of the operation.
-     */
-    Stopwatch stopwatch();
-
-    /**
-     * Compute checksum from arguments.
-     */
-    Long computeChecksum();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
deleted file mode 100644
index f3fc610..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.stats.BroadCastStatsLogger;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Encapsulate stream op stats construction to make it easier to access stream
- * op stats consistently from different scopes.
- */
-public class StreamOpStats {
-    private final StatsLogger baseStatsLogger;
-    private final StatsLogger requestStatsLogger;
-    private final StatsLogger recordsStatsLogger;
-    private final StatsLogger requestDeniedStatsLogger;
-    private final StatsLogger streamStatsLogger;
-
-    public StreamOpStats(StatsLogger statsLogger,
-                         StatsLogger perStreamStatsLogger) {
-        this.baseStatsLogger = statsLogger;
-        this.requestStatsLogger = statsLogger.scope("request");
-        this.recordsStatsLogger = statsLogger.scope("records");
-        this.requestDeniedStatsLogger = statsLogger.scope("denied");
-        this.streamStatsLogger = perStreamStatsLogger;
-    }
-
-    public StatsLogger baseStatsLogger(String opName) {
-        return baseStatsLogger;
-    }
-
-    public Counter baseCounter(String opName) {
-        return baseStatsLogger.getCounter(opName);
-    }
-
-    public StatsLogger baseScope(String opName) {
-        return baseStatsLogger.scope(opName);
-    }
-
-    public OpStatsLogger requestLatencyStat(String opName) {
-        return requestStatsLogger.getOpStatsLogger(opName);
-    }
-
-    public StatsLogger requestScope(String scopeName) {
-        return requestStatsLogger.scope(scopeName);
-    }
-
-    public Counter scopedRequestCounter(String opName, String counterName) {
-        return requestScope(opName).getCounter(counterName);
-    }
-
-    public Counter requestCounter(String counterName) {
-        return requestStatsLogger.getCounter(counterName);
-    }
-
-    public Counter requestPendingCounter(String counterName) {
-        return requestCounter(counterName);
-    }
-
-    public Counter requestDeniedCounter(String counterName) {
-        return requestDeniedStatsLogger.getCounter(counterName);
-    }
-
-    public Counter recordsCounter(String counterName) {
-        return recordsStatsLogger.getCounter(counterName);
-    }
-
-    public StatsLogger streamRequestStatsLogger(Partition partition) {
-        return BroadCastStatsLogger.masterslave(
-            streamStatsLogger.scope(partition.getStream()).scope("partition")
-                .scope(partition.getPaddedId()), streamStatsLogger.scope(partition.getStream())
-                .scope("aggregate"));
-    }
-
-    public StatsLogger streamRequestScope(Partition partition, String scopeName) {
-        return streamRequestStatsLogger(partition).scope(scopeName);
-    }
-
-    public OpStatsLogger streamRequestLatencyStat(Partition partition, String opName) {
-        return streamRequestStatsLogger(partition).getOpStatsLogger(opName);
-    }
-
-    public Counter streamRequestCounter(Partition partition, String opName, String counterName) {
-        return streamRequestScope(partition, opName).getCounter(counterName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
deleted file mode 100644
index 7a38d14..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.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 org.apache.distributedlog.service.stream;
-
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.RequestDeniedException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.ProtocolUtils;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Operation to truncate a log stream.
- */
-public class TruncateOp extends AbstractWriteOp {
-
-    private static final Logger logger = LoggerFactory.getLogger(TruncateOp.class);
-
-    private final Counter deniedTruncateCounter;
-    private final DLSN dlsn;
-    private final AccessControlManager accessControlManager;
-
-    public TruncateOp(String stream,
-                      DLSN dlsn,
-                      StatsLogger statsLogger,
-                      StatsLogger perStreamStatsLogger,
-                      Long checksum,
-                      Feature checksumDisabledFeature,
-                      AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "truncate"), checksum, checksumDisabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedTruncateCounter = streamOpStats.requestDeniedCounter("truncate");
-        this.accessControlManager = accessControlManager;
-        this.dlsn = dlsn;
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return ProtocolUtils.truncateOpCRC32(stream, dlsn);
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        if (!stream.equals(writer.getStreamName())) {
-            logger.error("Truncate: Stream Name Mismatch in the Stream Map {}, {}", stream, writer.getStreamName());
-            return Future.exception(new IllegalStateException("The stream mapping is incorrect, fail the request"));
-        }
-        return writer.truncate(dlsn).map(new AbstractFunction1<Boolean, WriteResponse>() {
-            @Override
-            public WriteResponse apply(Boolean v1) {
-                return ResponseUtils.writeSuccess();
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowTruncate(stream)) {
-            deniedTruncateCounter.inc();
-            throw new RequestDeniedException(stream, "truncate");
-        }
-        super.preExecute();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
deleted file mode 100644
index c4bdcc2..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.LogRecord;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.RequestDeniedException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.service.config.ServerConfiguration;
-import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.thrift.service.StatusCode;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.ProtocolUtils;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.nio.ByteBuffer;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-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.AbstractFunction1;
-
-/**
- * Operation to write a single record to a log stream.
- */
-public class WriteOp extends AbstractWriteOp implements WriteOpWithPayload {
-
-    private static final Logger logger = LoggerFactory.getLogger(WriteOp.class);
-
-    private final byte[] payload;
-    private final boolean isRecordSet;
-
-    // Stats
-    private final Counter deniedWriteCounter;
-    private final Counter successRecordCounter;
-    private final Counter failureRecordCounter;
-    private final Counter redirectRecordCounter;
-    private final OpStatsLogger latencyStat;
-    private final Counter bytes;
-    private final Counter writeBytes;
-
-    private final byte dlsnVersion;
-    private final AccessControlManager accessControlManager;
-
-    public WriteOp(String stream,
-                   ByteBuffer data,
-                   StatsLogger statsLogger,
-                   StatsLogger perStreamStatsLogger,
-                   StreamPartitionConverter streamPartitionConverter,
-                   ServerConfiguration conf,
-                   byte dlsnVersion,
-                   Long checksum,
-                   boolean isRecordSet,
-                   Feature checksumDisabledFeature,
-                   AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "write"), checksum, checksumDisabledFeature);
-        payload = new byte[data.remaining()];
-        data.get(payload);
-        this.isRecordSet = isRecordSet;
-
-        final Partition partition = streamPartitionConverter.convert(stream);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.successRecordCounter = streamOpStats.recordsCounter("success");
-        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
-        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
-        this.deniedWriteCounter = streamOpStats.requestDeniedCounter("write");
-        this.writeBytes = streamOpStats.scopedRequestCounter("write", "bytes");
-        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "write");
-        this.bytes = streamOpStats.streamRequestCounter(partition, "write", "bytes");
-
-        this.dlsnVersion = dlsnVersion;
-        this.accessControlManager = accessControlManager;
-
-        final long size = getPayloadSize();
-        result().addEventListener(new FutureEventListener<WriteResponse>() {
-            @Override
-            public void onSuccess(WriteResponse response) {
-                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
-                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                    bytes.add(size);
-                    writeBytes.add(size);
-                } else {
-                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                }
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-            }
-        });
-    }
-
-    @Override
-    public long getPayloadSize() {
-      return payload.length;
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return ProtocolUtils.writeOpCRC32(stream, payload);
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowWrite(stream)) {
-            deniedWriteCounter.inc();
-            throw new RequestDeniedException(stream, "write");
-        }
-        super.preExecute();
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        if (!stream.equals(writer.getStreamName())) {
-            logger.error("Write: Stream Name Mismatch in the Stream Map {}, {}", stream, writer.getStreamName());
-            return Future.exception(new IllegalStateException("The stream mapping is incorrect, fail the request"));
-        }
-
-        long txnId;
-        Future<DLSN> writeResult;
-        synchronized (txnLock) {
-            txnId = sequencer.nextId();
-            LogRecord record = new LogRecord(txnId, payload);
-            if (isRecordSet) {
-                record.setRecordSet();
-            }
-            writeResult = writer.write(record);
-        }
-        return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
-            @Override
-            public WriteResponse apply(DLSN value) {
-                successRecordCounter.inc();
-                return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
-            }
-        });
-    }
-
-    @Override
-    protected void fail(ResponseHeader header) {
-        if (StatusCode.FOUND == header.getCode()) {
-            redirectRecordCounter.inc();
-        } else {
-            failureRecordCounter.inc();
-        }
-        super.fail(header);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
deleted file mode 100644
index e411b420..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-/**
- * A write operation with payload.
- */
-public interface WriteOpWithPayload {
-
-    // Return the payload size in bytes
-    long getPayloadSize();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java
deleted file mode 100644
index fcaee35..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.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 org.apache.distributedlog.service.stream.admin;
-
-import org.apache.distributedlog.exceptions.DLException;
-import com.twitter.util.Future;
-
-/**
- * Admin operation interface.
- */
-public interface AdminOp<RespT> {
-
-    /**
-     * Invoked before the stream op is executed.
-     */
-    void preExecute() throws DLException;
-
-    /**
-     * Execute the operation.
-     *
-     * @return the future represents the response of the operation
-     */
-    Future<RespT> execute();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java
deleted file mode 100644
index 89a2566..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.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 org.apache.distributedlog.service.stream.admin;
-
-import static org.apache.distributedlog.service.stream.AbstractStreamOp.requestStat;
-
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.service.stream.StreamManager;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Operation to create log stream.
- */
-public class CreateOp extends StreamAdminOp {
-
-  public CreateOp(String stream,
-                  StatsLogger statsLogger,
-                  StreamManager streamManager,
-                  Long checksum,
-                  Feature checksumEnabledFeature) {
-    super(stream,
-            streamManager,
-            requestStat(statsLogger, "create"),
-            checksum,
-            checksumEnabledFeature);
-  }
-
-  @Override
-  protected Future<WriteResponse> executeOp() {
-    Future<Void> result = streamManager.createStreamAsync(stream);
-    return result.map(new AbstractFunction1<Void, WriteResponse>() {
-      @Override
-      public WriteResponse apply(Void value) {
-        return ResponseUtils.writeSuccess();
-      }
-    });
-  }
-}


[07/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
deleted file mode 100644
index 3c53ccf..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.distributedlog.service.DistributedLogClient;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A balancer balances ownerships between two targets.
- */
-public class SimpleBalancer implements Balancer {
-
-    private static final Logger logger = LoggerFactory.getLogger(SimpleBalancer.class);
-
-    protected final String target1;
-    protected final String target2;
-    protected final DistributedLogClient targetClient1;
-    protected final DistributedLogClient targetClient2;
-    protected final MonitorServiceClient targetMonitor1;
-    protected final MonitorServiceClient targetMonitor2;
-
-    public SimpleBalancer(String name1,
-                          DistributedLogClient client1,
-                          MonitorServiceClient monitor1,
-                          String name2,
-                          DistributedLogClient client2,
-                          MonitorServiceClient monitor2) {
-        this.target1 = name1;
-        this.targetClient1 = client1;
-        this.targetMonitor1 = monitor1;
-        this.target2 = name2;
-        this.targetClient2 = client2;
-        this.targetMonitor2 = monitor2;
-    }
-
-    protected static int countNumberStreams(Map<SocketAddress, Set<String>> distribution) {
-        int count = 0;
-        for (Set<String> streams : distribution.values()) {
-            count += streams.size();
-        }
-        return count;
-    }
-
-    @Override
-    public void balance(int rebalanceWaterMark,
-                        double rebalanceTolerancePercentage,
-                        int rebalanceConcurrency,
-                        Optional<RateLimiter> rebalanceRateLimiter) {
-        // get the ownership distributions from individual targets
-        Map<SocketAddress, Set<String>> distribution1 = targetMonitor1.getStreamOwnershipDistribution();
-        Map<SocketAddress, Set<String>> distribution2 = targetMonitor2.getStreamOwnershipDistribution();
-
-        // get stream counts
-        int proxyCount1 = distribution1.size();
-        int streamCount1 = countNumberStreams(distribution1);
-        int proxyCount2 = distribution2.size();
-        int streamCount2 = countNumberStreams(distribution2);
-
-        logger.info("'{}' has {} streams by {} proxies; while '{}' has {} streams by {} proxies.",
-                    new Object[] {target1, streamCount1, proxyCount1, target2, streamCount2, proxyCount2 });
-
-        String source, target;
-        Map<SocketAddress, Set<String>> srcDistribution;
-        DistributedLogClient srcClient, targetClient;
-        MonitorServiceClient srcMonitor, targetMonitor;
-        int srcStreamCount, targetStreamCount;
-        if (streamCount1 > streamCount2) {
-            source = target1;
-            srcStreamCount = streamCount1;
-            srcClient = targetClient1;
-            srcMonitor = targetMonitor1;
-            srcDistribution = distribution1;
-
-            target = target2;
-            targetStreamCount = streamCount2;
-            targetClient = targetClient2;
-            targetMonitor = targetMonitor2;
-        } else {
-            source = target2;
-            srcStreamCount = streamCount2;
-            srcClient = targetClient2;
-            srcMonitor = targetMonitor2;
-            srcDistribution = distribution2;
-
-            target = target1;
-            targetStreamCount = streamCount1;
-            targetClient = targetClient1;
-            targetMonitor = targetMonitor1;
-        }
-
-        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
-        loadDistribution.put(source, srcStreamCount);
-        loadDistribution.put(target, targetStreamCount);
-
-        // Calculate how many streams to be rebalanced from src region to target region
-        int numStreamsToRebalance = BalancerUtils.calculateNumStreamsToRebalance(
-            source, loadDistribution, rebalanceWaterMark, rebalanceTolerancePercentage);
-
-        if (numStreamsToRebalance <= 0) {
-            logger.info("No streams need to be rebalanced from '{}' to '{}'.", source, target);
-            return;
-        }
-
-        StreamChooser streamChooser =
-                LimitedStreamChooser.of(new CountBasedStreamChooser(srcDistribution), numStreamsToRebalance);
-        StreamMover streamMover =
-            new StreamMoverImpl(source, srcClient, srcMonitor, target, targetClient, targetMonitor);
-
-        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
-    }
-
-    @Override
-    public void balanceAll(String source,
-                           int rebalanceConcurrency,
-                           Optional<RateLimiter> rebalanceRateLimiter) {
-        String target;
-        DistributedLogClient sourceClient, targetClient;
-        MonitorServiceClient sourceMonitor, targetMonitor;
-        if (target1.equals(source)) {
-            sourceClient = targetClient1;
-            sourceMonitor = targetMonitor1;
-            target = target2;
-            targetClient = targetClient2;
-            targetMonitor = targetMonitor2;
-        } else if (target2.equals(source)) {
-            sourceClient = targetClient2;
-            sourceMonitor = targetMonitor2;
-            target = target1;
-            targetClient = targetClient1;
-            targetMonitor = targetMonitor1;
-        } else {
-            throw new IllegalArgumentException("Unknown target " + source);
-        }
-
-        // get the ownership distributions from individual targets
-        Map<SocketAddress, Set<String>> distribution = sourceMonitor.getStreamOwnershipDistribution();
-
-        if (distribution.isEmpty()) {
-            return;
-        }
-
-        StreamChooser streamChooser = new CountBasedStreamChooser(distribution);
-        StreamMover streamMover =
-            new StreamMoverImpl(source, sourceClient, sourceMonitor, target, targetClient, targetMonitor);
-
-        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
-    }
-
-    private void moveStreams(StreamChooser streamChooser,
-                             StreamMover streamMover,
-                             int concurrency,
-                             Optional<RateLimiter> rateLimiter) {
-        CountDownLatch doneLatch = new CountDownLatch(concurrency);
-        RegionMover regionMover = new RegionMover(streamChooser, streamMover, rateLimiter, doneLatch);
-        ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
-        try {
-            for (int i = 0; i < concurrency; i++) {
-                executorService.submit(regionMover);
-            }
-
-            try {
-                doneLatch.await();
-            } catch (InterruptedException e) {
-                logger.info("{} is interrupted. Stopping it ...", streamMover);
-                regionMover.shutdown();
-            }
-        } finally {
-            executorService.shutdown();
-        }
-
-    }
-
-    /**
-     * Move streams from <i>src</i> region to <i>target</i> region.
-     */
-    static class RegionMover implements Runnable {
-
-        final StreamChooser streamChooser;
-        final StreamMover streamMover;
-        final Optional<RateLimiter> rateLimiter;
-        final CountDownLatch doneLatch;
-        volatile boolean running = true;
-
-        RegionMover(StreamChooser streamChooser,
-                    StreamMover streamMover,
-                    Optional<RateLimiter> rateLimiter,
-                    CountDownLatch doneLatch) {
-            this.streamChooser = streamChooser;
-            this.streamMover = streamMover;
-            this.rateLimiter = rateLimiter;
-            this.doneLatch = doneLatch;
-        }
-
-        @Override
-        public void run() {
-            while (running) {
-                if (rateLimiter.isPresent()) {
-                    rateLimiter.get().acquire();
-                }
-
-                String stream = streamChooser.choose();
-                if (null == stream) {
-                    break;
-                }
-
-                streamMover.moveStream(stream);
-            }
-            doneLatch.countDown();
-        }
-
-        void shutdown() {
-            running = false;
-        }
-    }
-
-    @Override
-    public void close() {
-        // no-op
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
deleted file mode 100644
index 1d7b6f7..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-/**
- * Choose a stream to rebalance.
- */
-public interface StreamChooser {
-    /**
-     * Choose a stream to rebalance.
-     *
-     * @return stream chose
-     */
-    String choose();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java
deleted file mode 100644
index 4a04530..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.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 org.apache.distributedlog.service.balancer;
-
-/**
- * A stream mover to move streams between proxies.
- */
-public interface StreamMover {
-
-    /**
-     * Move given stream <i>streamName</i>.
-     *
-     * @param streamName
-     *          stream name to move
-     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
-     * @throws Exception
-     */
-    boolean moveStream(final String streamName);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
deleted file mode 100644
index 68d934b..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.distributedlog.service.DistributedLogClient;
-import com.twitter.util.Await;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Move Streams from <i>src</i> to <i>target</i>.
- */
-public class StreamMoverImpl implements StreamMover {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamMoverImpl.class);
-
-    final String source, target;
-    final DistributedLogClient srcClient, targetClient;
-    final MonitorServiceClient srcMonitor, targetMonitor;
-
-    public StreamMoverImpl(String source, DistributedLogClient srcClient, MonitorServiceClient srcMonitor,
-                           String target, DistributedLogClient targetClient, MonitorServiceClient targetMonitor) {
-        this.source = source;
-        this.srcClient = srcClient;
-        this.srcMonitor = srcMonitor;
-        this.target = target;
-        this.targetClient = targetClient;
-        this.targetMonitor = targetMonitor;
-    }
-
-    /**
-     * Move given stream <i>streamName</i>.
-     *
-     * @param streamName
-     *          stream name to move
-     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
-     * @throws Exception
-     */
-    public boolean moveStream(final String streamName) {
-        try {
-            doMoveStream(streamName);
-            return true;
-        } catch (Exception e) {
-            return false;
-        }
-    }
-
-    private void doMoveStream(final String streamName) throws Exception {
-        Await.result(srcClient.release(streamName).flatMap(new Function<Void, Future<Void>>() {
-            @Override
-            public Future<Void> apply(Void result) {
-                return targetMonitor.check(streamName).addEventListener(new FutureEventListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        logger.info("Moved stream {} from {} to {}.",
-                                new Object[]{streamName, source, target});
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        logger.info("Failed to move stream {} from region {} to {} : ",
-                                new Object[]{streamName, source, target, cause});
-                    }
-                });
-            }
-        }));
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("StreamMover('").append(source).append("' -> '").append(target).append("')");
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/package-info.java
deleted file mode 100644
index 9eb8950..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/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.
- */
-/**
- * Balancer to move streams around to balance the traffic.
- */
-package org.apache.distributedlog.service.balancer;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
deleted file mode 100644
index 7d72093..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.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 org.apache.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.ConcurrentConstConfiguration;
-import org.apache.distributedlog.config.ConfigurationSubscription;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.config.FileConfigurationBuilder;
-import org.apache.distributedlog.config.PropertiesConfigurationBuilder;
-import java.io.File;
-import java.net.MalformedURLException;
-import java.util.List;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * For all streams return the same dynamic config based on configFile.
- */
-public class DefaultStreamConfigProvider implements StreamConfigProvider {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DefaultStreamConfigProvider.class);
-
-    private final Optional<DynamicDistributedLogConfiguration> dynConf;
-    private final ConfigurationSubscription confSub;
-
-    public DefaultStreamConfigProvider(String configFilePath,
-                                       ScheduledExecutorService executorService,
-                                       int reloadPeriod,
-                                       TimeUnit reloadUnit)
-        throws ConfigurationException {
-        try {
-            File configFile = new File(configFilePath);
-            FileConfigurationBuilder properties =
-                new PropertiesConfigurationBuilder(configFile.toURI().toURL());
-            ConcurrentConstConfiguration defaultConf =
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration());
-            DynamicDistributedLogConfiguration conf =
-                new DynamicDistributedLogConfiguration(defaultConf);
-            List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(properties);
-            confSub = new ConfigurationSubscription(
-                conf, fileConfigBuilders, executorService, reloadPeriod, reloadUnit);
-            this.dynConf = Optional.of(conf);
-        } catch (MalformedURLException ex) {
-            throw new ConfigurationException(ex);
-        }
-    }
-
-    @Override
-    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
-        return dynConf;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java
deleted file mode 100644
index 195f29d..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.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 org.apache.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * For all streams return an absent configuration.
- */
-public class NullStreamConfigProvider implements StreamConfigProvider {
-    static final Logger LOG = LoggerFactory.getLogger(NullStreamConfigProvider.class);
-
-    private static final Optional<DynamicDistributedLogConfiguration> nullConf =
-            Optional.<DynamicDistributedLogConfiguration>absent();
-
-    @Override
-    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
-        return nullConf;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
deleted file mode 100644
index 257b4be..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
+++ /dev/null
@@ -1,443 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.config;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.CompositeConfiguration;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.SystemConfiguration;
-
-/**
- * Configuration for DistributedLog Server.
- */
-public class ServerConfiguration extends CompositeConfiguration {
-
-    private static ClassLoader defaultLoader;
-
-    static {
-        defaultLoader = Thread.currentThread().getContextClassLoader();
-        if (null == defaultLoader) {
-            defaultLoader = DistributedLogConfiguration.class.getClassLoader();
-        }
-    }
-
-    // Server DLSN version
-    protected static final String SERVER_DLSN_VERSION = "server_dlsn_version";
-    protected static final byte SERVER_DLSN_VERSION_DEFAULT = DLSN.VERSION1;
-
-    // Server Durable Write Enable/Disable Flag
-    protected static final String SERVER_DURABLE_WRITE_ENABLED = "server_durable_write_enabled";
-    protected static final boolean SERVER_DURABLE_WRITE_ENABLED_DEFAULT = true;
-
-    // Server Region Id
-    protected static final String SERVER_REGION_ID = "server_region_id";
-    protected static final int SERVER_REGION_ID_DEFAULT = DistributedLogConstants.LOCAL_REGION_ID;
-
-    // Server Port
-    protected static final String SERVER_PORT = "server_port";
-    protected static final int SERVER_PORT_DEFAULT = 0;
-
-    // Server Shard Id
-    protected static final String SERVER_SHARD_ID = "server_shard";
-    protected static final int SERVER_SHARD_ID_DEFAULT = -1;
-
-    // Server Threads
-    protected static final String SERVER_NUM_THREADS = "server_threads";
-    protected static final int SERVER_NUM_THREADS_DEFAULT = Runtime.getRuntime().availableProcessors();
-
-    // Server enable per stream stat
-    protected static final String SERVER_ENABLE_PERSTREAM_STAT = "server_enable_perstream_stat";
-    protected static final boolean SERVER_ENABLE_PERSTREAM_STAT_DEFAULT = true;
-
-    // Server graceful shutdown period (in millis)
-    protected static final String SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS = "server_graceful_shutdown_period_ms";
-    protected static final long SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT = 0L;
-
-    // Server service timeout
-    public static final String SERVER_SERVICE_TIMEOUT_MS = "server_service_timeout_ms";
-    public static final String SERVER_SERVICE_TIMEOUT_MS_OLD = "serviceTimeoutMs";
-    public static final long SERVER_SERVICE_TIMEOUT_MS_DEFAULT = 0;
-
-    // Server close writer timeout
-    public static final String SERVER_WRITER_CLOSE_TIMEOUT_MS = "server_writer_close_timeout_ms";
-    public static final long SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT = 1000;
-
-    // Server stream probation timeout
-    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS = "server_stream_probation_timeout_ms";
-    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD = "streamProbationTimeoutMs";
-    public static final long SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT = 60 * 1000 * 5;
-
-    // Server stream to partition converter
-    protected static final String SERVER_STREAM_PARTITION_CONVERTER_CLASS = "stream_partition_converter_class";
-
-    // Use hostname as the allocator pool name
-    protected static final String SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME =
-        "server_use_hostname_as_allocator_pool_name";
-    protected static final boolean SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT = false;
-    //Configure refresh interval for calculating resource placement in seconds
-    public static final String SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S =
-        "server_resource_placement_refresh_interval_sec";
-    public static final int  SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT = 120;
-
-    public ServerConfiguration() {
-        super();
-        addConfiguration(new SystemConfiguration());
-    }
-
-    /**
-     * Load configurations from {@link DistributedLogConfiguration}.
-     *
-     * @param dlConf
-     *          distributedlog configuration
-     */
-    public void loadConf(DistributedLogConfiguration dlConf) {
-        addConfiguration(dlConf);
-    }
-
-    /**
-     * Set the version to encode dlsn.
-     *
-     * @param version
-     *          dlsn version
-     * @return server configuration
-     */
-    public ServerConfiguration setDlsnVersion(byte version) {
-        setProperty(SERVER_DLSN_VERSION, version);
-        return this;
-    }
-
-    /**
-     * Get the version to encode dlsn.
-     *
-     * @see DLSN
-     * @return version to encode dlsn.
-     */
-    public byte getDlsnVersion() {
-        return getByte(SERVER_DLSN_VERSION, SERVER_DLSN_VERSION_DEFAULT);
-    }
-
-    /**
-     * Set the flag to enable/disable durable write.
-     *
-     * @param enabled
-     *          flag to enable/disable durable write
-     * @return server configuration
-     */
-    public ServerConfiguration enableDurableWrite(boolean enabled) {
-        setProperty(SERVER_DURABLE_WRITE_ENABLED, enabled);
-        return this;
-    }
-
-    /**
-     * Is durable write enabled.
-     *
-     * @return true if waiting writes to be durable. otherwise false.
-     */
-    public boolean isDurableWriteEnabled() {
-        return getBoolean(SERVER_DURABLE_WRITE_ENABLED, SERVER_DURABLE_WRITE_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Set the region id used to instantiate DistributedLogNamespace.
-     *
-     * @param regionId
-     *          region id
-     * @return server configuration
-     */
-    public ServerConfiguration setRegionId(int regionId) {
-        setProperty(SERVER_REGION_ID, regionId);
-        return this;
-    }
-
-    /**
-     * Get the region id used to instantiate {@link org.apache.distributedlog.namespace.DistributedLogNamespace}.
-     *
-     * @return region id used to instantiate DistributedLogNamespace
-     */
-    public int getRegionId() {
-        return getInt(SERVER_REGION_ID, SERVER_REGION_ID_DEFAULT);
-    }
-
-    /**
-     * Set the server port running for this service.
-     *
-     * @param port
-     *          server port
-     * @return server configuration
-     */
-    public ServerConfiguration setServerPort(int port) {
-        setProperty(SERVER_PORT, port);
-        return this;
-    }
-
-    /**
-     * Get the server port running for this service.
-     *
-     * @return server port
-     */
-    public int getServerPort() {
-        return getInt(SERVER_PORT, SERVER_PORT_DEFAULT);
-    }
-
-    /**
-     * Set the shard id of this server.
-     *
-     * @param shardId
-     *          shard id
-     * @return shard id of this server
-     */
-    public ServerConfiguration setServerShardId(int shardId) {
-        setProperty(SERVER_SHARD_ID, shardId);
-        return this;
-    }
-
-    /**
-     * Get the shard id of this server.
-     *
-     * <p>It would be used to instantiate the client id used for DistributedLogNamespace.
-     *
-     * @return shard id of this server.
-     */
-    public int getServerShardId() {
-        return getInt(SERVER_SHARD_ID, SERVER_SHARD_ID_DEFAULT);
-    }
-
-    /**
-     * Get the number of threads for the executor of this server.
-     *
-     * @return number of threads for the executor running in this server.
-     */
-    public int getServerThreads() {
-        return getInt(SERVER_NUM_THREADS, SERVER_NUM_THREADS_DEFAULT);
-    }
-
-    /**
-     * Set the number of threads for the executor of this server.
-     *
-     * @param numThreads
-     *          number of threads for the executor running in this server.
-     * @return server configuration
-     */
-    public ServerConfiguration setServerThreads(int numThreads) {
-        setProperty(SERVER_NUM_THREADS, numThreads);
-        return this;
-    }
-
-    /**
-     * Enable/Disable per stream stat.
-     *
-     * @param enabled
-     *          flag to enable/disable per stream stat
-     * @return server configuration
-     */
-    public ServerConfiguration setPerStreamStatEnabled(boolean enabled) {
-        setProperty(SERVER_ENABLE_PERSTREAM_STAT, enabled);
-        return this;
-    }
-
-    /**
-     * Whether the per stream stat enabled for not in this server.
-     *
-     * @return true if per stream stat enable, otherwise false.
-     */
-    public boolean isPerStreamStatEnabled() {
-        return getBoolean(SERVER_ENABLE_PERSTREAM_STAT, SERVER_ENABLE_PERSTREAM_STAT_DEFAULT);
-    }
-
-    /**
-     * Set the graceful shutdown period in millis.
-     *
-     * @param periodMs
-     *          graceful shutdown period in millis.
-     * @return server configuration
-     */
-    public ServerConfiguration setGracefulShutdownPeriodMs(long periodMs) {
-        setProperty(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, periodMs);
-        return this;
-    }
-
-    /**
-     * Get the graceful shutdown period in millis.
-     *
-     * @return graceful shutdown period in millis.
-     */
-    public long getGracefulShutdownPeriodMs() {
-        return getLong(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT);
-    }
-
-    /**
-     * Get timeout for stream op execution in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @return timeout for stream operation in proxy layer.
-     */
-    public long getServiceTimeoutMs() {
-        return getLong(SERVER_SERVICE_TIMEOUT_MS,
-                getLong(SERVER_SERVICE_TIMEOUT_MS_OLD, SERVER_SERVICE_TIMEOUT_MS_DEFAULT));
-    }
-
-    /**
-     * Set timeout for stream op execution in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @param timeoutMs
-     *          timeout for stream operation in proxy layer.
-     * @return dl configuration.
-     */
-    public ServerConfiguration setServiceTimeoutMs(long timeoutMs) {
-        setProperty(SERVER_SERVICE_TIMEOUT_MS, timeoutMs);
-        return this;
-    }
-
-    /**
-     * Get timeout for closing writer in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @return timeout for closing writer in proxy layer.
-     */
-    public long getWriterCloseTimeoutMs() {
-        return getLong(SERVER_WRITER_CLOSE_TIMEOUT_MS, SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT);
-    }
-
-    /**
-     * Set timeout for closing writer in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @param timeoutMs
-     *          timeout for closing writer in proxy layer.
-     * @return dl configuration.
-     */
-    public ServerConfiguration setWriterCloseTimeoutMs(long timeoutMs) {
-        setProperty(SERVER_WRITER_CLOSE_TIMEOUT_MS, timeoutMs);
-        return this;
-    }
-
-    /**
-     * How long should stream be kept in cache in probationary state after service timeout.
-     *
-     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
-     *
-     * @return stream probation timeout in ms.
-     */
-    public long getStreamProbationTimeoutMs() {
-        return getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS,
-                getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD, SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT));
-    }
-
-    /**
-     * How long should stream be kept in cache in probationary state after service timeout.
-     *
-     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
-     *
-     * @param timeoutMs probation timeout in ms.
-     * @return server configuration
-     */
-    public ServerConfiguration setStreamProbationTimeoutMs(long timeoutMs) {
-        setProperty(SERVER_STREAM_PROBATION_TIMEOUT_MS, timeoutMs);
-        return this;
-    }
-
-    /**
-     * Set the stream partition converter class.
-     *
-     * @param converterClass
-     *          stream partition converter class
-     * @return server configuration
-     */
-    public ServerConfiguration setStreamPartitionConverterClass(
-        Class<? extends StreamPartitionConverter> converterClass) {
-        setProperty(SERVER_STREAM_PARTITION_CONVERTER_CLASS, converterClass.getName());
-        return this;
-    }
-
-    /**
-     * Get the stream partition converter class.
-     *
-     * @return the stream partition converter class.
-     * @throws ConfigurationException
-     */
-    public Class<? extends StreamPartitionConverter> getStreamPartitionConverterClass()
-            throws ConfigurationException {
-        return ReflectionUtils.getClass(
-                this,
-                SERVER_STREAM_PARTITION_CONVERTER_CLASS,
-                IdentityStreamPartitionConverter.class,
-                StreamPartitionConverter.class,
-                defaultLoader);
-    }
-
-     /**
-      * Set if use hostname as the allocator pool name.
-      *
-      * @param useHostname whether to use hostname as the allocator pool name.
-      * @return server configuration
-      * @see #isUseHostnameAsAllocatorPoolName()
-      */
-    public ServerConfiguration setUseHostnameAsAllocatorPoolName(boolean useHostname) {
-        setProperty(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME, useHostname);
-        return this;
-    }
-
-    /**
-     * Get if use hostname as the allocator pool name.
-     *
-     * @return true if use hostname as the allocator pool name. otherwise, use
-     * {@link #getServerShardId()} as the allocator pool name.
-     * @see #getServerShardId()
-     */
-    public boolean isUseHostnameAsAllocatorPoolName() {
-        return getBoolean(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME,
-            SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT);
-    }
-
-    public ServerConfiguration setResourcePlacementRefreshInterval(int refreshIntervalSecs) {
-        setProperty(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, refreshIntervalSecs);
-        return this;
-    }
-
-    public int getResourcePlacementRefreshInterval() {
-        return getInt(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT);
-    }
-
-    /**
-     * Validate the configuration.
-     *
-     * @throws IllegalStateException when there are any invalid settings.
-     */
-    public void validate() {
-        byte dlsnVersion = getDlsnVersion();
-        checkArgument(dlsnVersion >= DLSN.VERSION0 && dlsnVersion <= DLSN.VERSION1,
-                "Unknown dlsn version " + dlsnVersion);
-        checkArgument(getServerThreads() > 0,
-                "Invalid number of server threads : " + getServerThreads());
-        checkArgument(getServerShardId() >= 0,
-                "Invalid server shard id : " + getServerShardId());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java
deleted file mode 100644
index 29052f9..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.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 org.apache.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-import org.apache.distributedlog.config.DynamicConfigurationFactory;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import java.io.File;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Provide per stream configuration to DistributedLog service layer.
- */
-public class ServiceStreamConfigProvider implements StreamConfigProvider {
-
-    private static final Logger LOG = LoggerFactory.getLogger(ServiceStreamConfigProvider.class);
-
-    private static final String CONFIG_EXTENSION = "conf";
-
-    private final File configBaseDir;
-    private final File defaultConfigFile;
-    private final StreamPartitionConverter partitionConverter;
-    private final DynamicConfigurationFactory configFactory;
-    private final DynamicDistributedLogConfiguration defaultDynConf;
-
-    public ServiceStreamConfigProvider(String configPath,
-                                       String defaultConfigPath,
-                                       StreamPartitionConverter partitionConverter,
-                                       ScheduledExecutorService executorService,
-                                       int reloadPeriod,
-                                       TimeUnit reloadUnit)
-                                       throws ConfigurationException {
-        this.configBaseDir = new File(configPath);
-        if (!configBaseDir.exists()) {
-            throw new ConfigurationException("Stream configuration base directory "
-                + configPath + " does not exist");
-        }
-        this.defaultConfigFile = new File(configPath);
-        if (!defaultConfigFile.exists()) {
-            throw new ConfigurationException("Stream configuration default config "
-                + defaultConfigPath + " does not exist");
-        }
-
-        // Construct reloading default configuration
-        this.partitionConverter = partitionConverter;
-        this.configFactory = new DynamicConfigurationFactory(executorService, reloadPeriod, reloadUnit);
-        // We know it exists from the check above.
-        this.defaultDynConf = configFactory.getDynamicConfiguration(defaultConfigPath).get();
-    }
-
-    @Override
-    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
-        String configName = partitionConverter.convert(streamName).getStream();
-        String configPath = getConfigPath(configName);
-        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.<DynamicDistributedLogConfiguration>absent();
-        try {
-            dynConf = configFactory.getDynamicConfiguration(configPath, defaultDynConf);
-        } catch (ConfigurationException ex) {
-            LOG.warn("Configuration exception for stream {} ({}) at {}",
-                    new Object[] {streamName, configName, configPath, ex});
-        }
-        return dynConf;
-    }
-
-    private String getConfigPath(String configName) {
-        return new File(configBaseDir, String.format("%s.%s", configName, CONFIG_EXTENSION)).getPath();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java
deleted file mode 100644
index c704f70..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.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 org.apache.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-
-/**
- * Expose per-stream configs to dl proxy.
- */
-public interface StreamConfigProvider {
-    /**
-     * Get dynamic per stream config overrides for a given stream.
-     *
-     * @param streamName stream name to return config for
-     * @return Optional dynamic configuration instance
-     */
-    Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/package-info.java
deleted file mode 100644
index b07605e..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/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 Server Configurations.
- */
-package org.apache.distributedlog.service.config;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/package-info.java
deleted file mode 100644
index 3fcfeda..0000000
--- a/distributedlog-service/src/main/java/org/apache/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 Proxy Service.
- */
-package org.apache.distributedlog.service;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java
deleted file mode 100644
index fa3dd49..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import com.twitter.util.Future;
-
-/**
- * Equal Load Appraiser.
- *
- * <p>Created for those who hold these truths to be self-evident, that all streams are created equal,
- * that they are endowed by their creator with certain unalienable loads, that among these are
- * Uno, Eins, and One.
- */
-public class EqualLoadAppraiser implements LoadAppraiser {
-    @Override
-    public Future<StreamLoad> getStreamLoad(String stream) {
-        return Future.value(new StreamLoad(stream, 1));
-    }
-
-    @Override
-    public Future<Void> refreshCache() {
-        return Future.value(null);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
deleted file mode 100644
index 2e9dd6b..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.Futures;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-/**
- * Least Load Placement Policy.
- *
- * <p>A LoadPlacementPolicy that attempts to place streams in such a way that the load is balanced as
- * evenly as possible across all shards. The LoadAppraiser remains responsible for determining what
- * the load of a server would be. This placement policy then distributes these streams across the
- * servers.
- */
-public class LeastLoadPlacementPolicy extends PlacementPolicy {
-
-    private static final Logger logger = LoggerFactory.getLogger(LeastLoadPlacementPolicy.class);
-
-    private TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
-    private Map<String, String> streamToServer = new HashMap<String, String>();
-
-    public LeastLoadPlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
-                                    DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
-                                    Duration refreshInterval, StatsLogger statsLogger) {
-        super(loadAppraiser, routingService, namespace, placementStateManager, refreshInterval, statsLogger);
-        statsLogger.registerGauge("placement/load.diff", new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                if (serverLoads.size() > 0) {
-                    return serverLoads.last().getLoad() - serverLoads.first().getLoad();
-                } else {
-                    return getDefaultValue();
-                }
-            }
-        });
-    }
-
-    private synchronized String getStreamOwner(String stream) {
-        return streamToServer.get(stream);
-    }
-
-    @Override
-    public Future<String> placeStream(String stream) {
-        String streamOwner = getStreamOwner(stream);
-        if (null != streamOwner) {
-            return Future.value(streamOwner);
-        }
-        Future<StreamLoad> streamLoadFuture = loadAppraiser.getStreamLoad(stream);
-        return streamLoadFuture.map(new Function<StreamLoad, String>() {
-            @Override
-            public String apply(StreamLoad streamLoad) {
-                return placeStreamSynchronized(streamLoad);
-            }
-        });
-    }
-
-    private synchronized String placeStreamSynchronized(StreamLoad streamLoad) {
-        ServerLoad serverLoad = serverLoads.pollFirst();
-        serverLoad.addStream(streamLoad);
-        serverLoads.add(serverLoad);
-        return serverLoad.getServer();
-    }
-
-    @Override
-    public void refresh() {
-        logger.info("Refreshing server loads.");
-        Future<Void> refresh = loadAppraiser.refreshCache();
-        final Set<String> servers = getServers();
-        final Set<String> allStreams = getStreams();
-        Future<TreeSet<ServerLoad>> serverLoadsFuture = refresh.flatMap(
-            new Function<Void, Future<TreeSet<ServerLoad>>>() {
-            @Override
-            public Future<TreeSet<ServerLoad>> apply(Void v1) {
-                return calculate(servers, allStreams);
-            }
-        });
-        serverLoadsFuture.map(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
-                try {
-                    updateServerLoads(serverLoads);
-                } catch (PlacementStateManager.StateManagerSaveException e) {
-                    logger.error("The refreshed mapping could not be persisted and will not be used.", e);
-                }
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    private synchronized void updateServerLoads(TreeSet<ServerLoad> serverLoads)
-        throws PlacementStateManager.StateManagerSaveException {
-        this.placementStateManager.saveOwnership(serverLoads);
-        this.streamToServer = serverLoadsToMap(serverLoads);
-        this.serverLoads = serverLoads;
-    }
-
-    @Override
-    public synchronized void load(TreeSet<ServerLoad> serverLoads) {
-        this.serverLoads = serverLoads;
-        this.streamToServer = serverLoadsToMap(serverLoads);
-    }
-
-    public Future<TreeSet<ServerLoad>> calculate(final Set<String> servers, Set<String> streams) {
-        logger.info("Calculating server loads");
-        final long startTime = System.currentTimeMillis();
-        ArrayList<Future<StreamLoad>> futures = new ArrayList<Future<StreamLoad>>(streams.size());
-
-        for (String stream : streams) {
-            Future<StreamLoad> streamLoad = loadAppraiser.getStreamLoad(stream);
-            futures.add(streamLoad);
-        }
-
-        return Futures.collect(futures).map(new Function<List<StreamLoad>, TreeSet<ServerLoad>>() {
-            @Override
-            public TreeSet<ServerLoad> apply(List<StreamLoad> streamLoads) {
-        /* Sort streamLoads so largest streams are placed first for better balance */
-                TreeSet<StreamLoad> streamQueue = new TreeSet<StreamLoad>();
-                for (StreamLoad streamLoad : streamLoads) {
-                    streamQueue.add(streamLoad);
-                }
-
-                TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
-                for (String server : servers) {
-                    ServerLoad serverLoad = new ServerLoad(server);
-                    if (!streamQueue.isEmpty()) {
-                        serverLoad.addStream(streamQueue.pollFirst());
-                    }
-                    serverLoads.add(serverLoad);
-                }
-
-                while (!streamQueue.isEmpty()) {
-                    ServerLoad serverLoad = serverLoads.pollFirst();
-                    serverLoad.addStream(streamQueue.pollFirst());
-                    serverLoads.add(serverLoad);
-                }
-                return serverLoads;
-            }
-        }).onSuccess(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
-                placementCalcStats.registerSuccessfulEvent(System.currentTimeMillis() - startTime);
-                return BoxedUnit.UNIT;
-            }
-        }).onFailure(new Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable t) {
-                logger.error("Failure calculating loads", t);
-                placementCalcStats.registerFailedEvent(System.currentTimeMillis() - startTime);
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    private static Map<String, String> serverLoadsToMap(Collection<ServerLoad> serverLoads) {
-        HashMap<String, String> streamToServer = new HashMap<String, String>(serverLoads.size());
-        for (ServerLoad serverLoad : serverLoads) {
-            for (StreamLoad streamLoad : serverLoad.getStreamLoads()) {
-                streamToServer.put(streamLoad.getStream(), serverLoad.getServer());
-            }
-        }
-        return streamToServer;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java
deleted file mode 100644
index 5cd8980..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.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 org.apache.distributedlog.service.placement;
-
-import com.twitter.util.Future;
-
-/**
- * Interface for load appraiser.
- */
-public interface LoadAppraiser {
-    /**
-     * Retrieve the stream load for a given {@code stream}.
-     *
-     * @param stream name of the stream
-     * @return the stream load of the stream.
-     */
-    Future<StreamLoad> getStreamLoad(String stream);
-
-    /**
-     * Refesch the cache.
-     * @return
-     */
-    Future<Void> refreshCache();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
deleted file mode 100644
index ac952aa..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.service.DLSocketAddress;
-import com.twitter.util.Duration;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.ScheduledThreadPoolTimer;
-import com.twitter.util.Time;
-import com.twitter.util.Timer;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.TreeSet;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-/**
- * A PlacementPolicy assigns streams to servers given an appraisal of the load that the stream contains.
- *
- * <p>The load of a stream is determined by the LoadAppraiser used. The PlacementPolicy will
- * then distributed these StreamLoads to the available servers in a manner defined by the
- * implementation creating ServerLoad objects. It then saves this assignment via the
- * PlacementStateManager.
- */
-public abstract class PlacementPolicy {
-
-    private static final Logger logger = LoggerFactory.getLogger(PlacementPolicy.class);
-
-    protected final LoadAppraiser loadAppraiser;
-    protected final RoutingService routingService;
-    protected final DistributedLogNamespace namespace;
-    protected final PlacementStateManager placementStateManager;
-    private final Duration refreshInterval;
-    protected final OpStatsLogger placementCalcStats;
-    private Timer placementRefreshTimer;
-
-    public PlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
-                           DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
-                           Duration refreshInterval, StatsLogger statsLogger) {
-        this.loadAppraiser = loadAppraiser;
-        this.routingService = routingService;
-        this.namespace = namespace;
-        this.placementStateManager = placementStateManager;
-        this.refreshInterval = refreshInterval;
-        placementCalcStats = statsLogger.getOpStatsLogger("placement");
-    }
-
-    public Set<String> getServers() {
-        Set<SocketAddress> hosts = routingService.getHosts();
-        Set<String> servers = new HashSet<String>(hosts.size());
-        for (SocketAddress address : hosts) {
-            servers.add(DLSocketAddress.toString((InetSocketAddress) address));
-        }
-        return servers;
-    }
-
-    public Set<String> getStreams() {
-        Set<String> streams = new HashSet<String>();
-        try {
-            Iterator<String> logs = namespace.getLogs();
-            while (logs.hasNext()) {
-                streams.add(logs.next());
-            }
-        } catch (IOException e) {
-            logger.error("Could not get streams for placement policy.", e);
-        }
-        return streams;
-    }
-
-    public void start(boolean leader) {
-        logger.info("Starting placement policy");
-
-        TreeSet<ServerLoad> emptyServerLoads = new TreeSet<ServerLoad>();
-        for (String server : getServers()) {
-            emptyServerLoads.add(new ServerLoad(server));
-        }
-        load(emptyServerLoads); //Pre-Load so streams don't NPE
-        if (leader) { //this is the leader shard
-            logger.info("Shard is leader. Scheduling timed refresh.");
-            placementRefreshTimer = new ScheduledThreadPoolTimer(1, "timer", true);
-            placementRefreshTimer.schedule(Time.now(), refreshInterval, new Function0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
-                    refresh();
-                    return BoxedUnit.UNIT;
-                }
-            });
-        } else {
-            logger.info("Shard is not leader. Watching for server load changes.");
-            placementStateManager.watch(new PlacementStateManager.PlacementCallback() {
-                @Override
-                public void callback(TreeSet<ServerLoad> serverLoads) {
-                    if (!serverLoads.isEmpty()) {
-                        load(serverLoads);
-                    }
-                }
-            });
-        }
-    }
-
-    public void close() {
-        if (placementRefreshTimer != null) {
-            placementRefreshTimer.stop();
-        }
-    }
-
-    /**
-     * Places the stream on a server according to the policy.
-     *
-     * <p>It returns a future containing the host that owns the stream upon completion
-     */
-    public abstract Future<String> placeStream(String stream);
-
-    /**
-     * Recalculates the entire placement mapping and updates stores it using the PlacementStateManager.
-     */
-    public abstract void refresh();
-
-    /**
-     * Loads the placement mapping into the node from a TreeSet of ServerLoads.
-     */
-    public abstract void load(TreeSet<ServerLoad> serverLoads);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
deleted file mode 100644
index 0187bed..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import java.util.TreeSet;
-
-/**
- * The PlacementStateManager handles persistence of calculated resource placements.
- */
-public interface PlacementStateManager {
-
-    /**
-     * Saves the ownership mapping as a TreeSet of ServerLoads to persistent storage.
-     */
-    void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException;
-
-    /**
-     * Loads the ownership mapping as TreeSet of ServerLoads from persistent storage.
-     */
-    TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException;
-
-    /**
-     * Watch the persistent storage for changes to the ownership mapping.
-     *
-     * <p>The placementCallback callbacks will be triggered with the new mapping when a change occurs.
-     */
-    void watch(PlacementCallback placementCallback);
-
-    /**
-     * Placement Callback.
-     *
-     * <p>The callback is triggered when server loads are updated.
-     */
-    interface PlacementCallback {
-        void callback(TreeSet<ServerLoad> serverLoads);
-    }
-
-    /**
-     * The base exception thrown when state manager encounters errors.
-     */
-    abstract class StateManagerException extends Exception {
-        public StateManagerException(String message, Exception e) {
-            super(message, e);
-        }
-    }
-
-    /**
-     * Exception thrown when failed to load the ownership mapping.
-     */
-    class StateManagerLoadException extends StateManagerException {
-        public StateManagerLoadException(Exception e) {
-            super("Load of Ownership failed", e);
-        }
-    }
-
-    /**
-     * Exception thrown when failed to save the ownership mapping.
-     */
-    class StateManagerSaveException extends StateManagerException {
-        public StateManagerSaveException(Exception e) {
-            super("Save of Ownership failed", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java
deleted file mode 100644
index d65c401..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.transport.TMemoryBuffer;
-import org.apache.thrift.transport.TMemoryInputTransport;
-
-/**
- * An object represents the server load.
- *
- * <p>A comparable data object containing the identifier of the server, total appraised load on the
- * server, and all streams assigned to the server by the resource placement mapping. This is
- * comparable first by load and then by server so that a sorted data structure of these will be
- * consistent across multiple calculations.
- */
-public class ServerLoad implements Comparable {
-    private static final int BUFFER_SIZE = 4096000;
-    private final String server;
-    private final HashSet<StreamLoad> streamLoads = new HashSet<StreamLoad>();
-    private long load = 0L;
-
-    public ServerLoad(String server) {
-        this.server = server;
-    }
-
-    public synchronized long addStream(StreamLoad stream) {
-        this.load += stream.getLoad();
-        streamLoads.add(stream);
-        return this.load;
-    }
-
-    public synchronized long removeStream(String stream) {
-        for (StreamLoad streamLoad : streamLoads) {
-            if (streamLoad.stream.equals(stream)) {
-                this.load -= streamLoad.getLoad();
-                streamLoads.remove(streamLoad);
-                return this.load;
-            }
-        }
-        return this.load; //Throwing an exception wouldn't help us as our logic should never reach here
-    }
-
-    public synchronized long getLoad() {
-        return load;
-    }
-
-    public synchronized Set<StreamLoad> getStreamLoads() {
-        return streamLoads;
-    }
-
-    public synchronized String getServer() {
-        return server;
-    }
-
-    protected synchronized org.apache.distributedlog.service.placement.thrift.ServerLoad toThrift() {
-        org.apache.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
-            new org.apache.distributedlog.service.placement.thrift.ServerLoad();
-        tServerLoad.setServer(server);
-        tServerLoad.setLoad(load);
-        ArrayList<org.apache.distributedlog.service.placement.thrift.StreamLoad> tStreamLoads =
-            new ArrayList<org.apache.distributedlog.service.placement.thrift.StreamLoad>();
-        for (StreamLoad streamLoad : streamLoads) {
-            tStreamLoads.add(streamLoad.toThrift());
-        }
-        tServerLoad.setStreams(tStreamLoads);
-        return tServerLoad;
-    }
-
-    public byte[] serialize() throws IOException {
-        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            toThrift().write(protocol);
-            transport.flush();
-            return transport.toString(UTF_8.name()).getBytes(UTF_8);
-        } catch (TException e) {
-            throw new IOException("Failed to serialize server load : ", e);
-        } catch (UnsupportedEncodingException uee) {
-            throw new IOException("Failed to serialize server load : ", uee);
-        }
-    }
-
-    public static ServerLoad deserialize(byte[] data) throws IOException {
-        org.apache.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
-            new org.apache.distributedlog.service.placement.thrift.ServerLoad();
-        TMemoryInputTransport transport = new TMemoryInputTransport(data);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            tServerLoad.read(protocol);
-            ServerLoad serverLoad = new ServerLoad(tServerLoad.getServer());
-            if (tServerLoad.isSetStreams()) {
-                for (org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad :
-                    tServerLoad.getStreams()) {
-                    serverLoad.addStream(new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad()));
-                }
-            }
-            return serverLoad;
-        } catch (TException e) {
-            throw new IOException("Failed to deserialize server load : ", e);
-        }
-    }
-
-    @Override
-    public synchronized int compareTo(Object o) {
-        ServerLoad other = (ServerLoad) o;
-        if (load == other.getLoad()) {
-            return server.compareTo(other.getServer());
-        } else {
-            return Long.compare(load, other.getLoad());
-        }
-    }
-
-    @Override
-    public synchronized boolean equals(Object o) {
-        if (!(o instanceof ServerLoad)) {
-            return false;
-        }
-        ServerLoad other = (ServerLoad) o;
-        return server.equals(other.getServer())
-            && load == other.getLoad()
-            && streamLoads.equals(other.getStreamLoads());
-    }
-
-    @Override
-    public synchronized String toString() {
-        return String.format("ServerLoad<Server: %s, Load: %d, Streams: %s>", server, load, streamLoads);
-    }
-
-    @Override
-    public synchronized int hashCode() {
-        return new HashCodeBuilder().append(server).append(load).append(streamLoads).build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java
deleted file mode 100644
index f271222..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.transport.TMemoryBuffer;
-import org.apache.thrift.transport.TMemoryInputTransport;
-
-/**
- * An object represent the load of a stream.
- *
- * <p>A comparable data object containing the identifier of the stream and the appraised load produced
- * by the stream.
- */
-public class StreamLoad implements Comparable {
-    private static final int BUFFER_SIZE = 4096;
-    public final String stream;
-    private final int load;
-
-    public StreamLoad(String stream, int load) {
-        this.stream = stream;
-        this.load = load;
-    }
-
-    public int getLoad() {
-        return load;
-    }
-
-    public String getStream() {
-        return stream;
-    }
-
-    protected org.apache.distributedlog.service.placement.thrift.StreamLoad toThrift() {
-        org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
-            new org.apache.distributedlog.service.placement.thrift.StreamLoad();
-        return tStreamLoad.setStream(stream).setLoad(load);
-    }
-
-    public byte[] serialize() throws IOException {
-        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            toThrift().write(protocol);
-            transport.flush();
-            return transport.toString(UTF_8.name()).getBytes(UTF_8);
-        } catch (TException e) {
-            throw new IOException("Failed to serialize stream load : ", e);
-        } catch (UnsupportedEncodingException uee) {
-            throw new IOException("Failed to serialize stream load : ", uee);
-        }
-    }
-
-    public static StreamLoad deserialize(byte[] data) throws IOException {
-        org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
-            new org.apache.distributedlog.service.placement.thrift.StreamLoad();
-        TMemoryInputTransport transport = new TMemoryInputTransport(data);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            tStreamLoad.read(protocol);
-            return new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad());
-        } catch (TException e) {
-            throw new IOException("Failed to deserialize stream load : ", e);
-        }
-    }
-
-    @Override
-    public int compareTo(Object o) {
-        StreamLoad other = (StreamLoad) o;
-        if (load == other.getLoad()) {
-            return stream.compareTo(other.getStream());
-        } else {
-            return Long.compare(load, other.getLoad());
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof StreamLoad)) {
-            return false;
-        }
-        StreamLoad other = (StreamLoad) o;
-        return stream.equals(other.getStream()) && load == other.getLoad();
-    }
-
-    @Override
-    public String toString() {
-        return String.format("StreamLoad<Stream: %s, Load: %d>", stream, load);
-    }
-
-    @Override
-    public int hashCode() {
-        return new HashCodeBuilder().append(stream).append(load).build();
-    }
-}


[03/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
deleted file mode 100644
index 58b5b2a..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Sets;
-import org.apache.distributedlog.DLMTestUtil;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.client.DistributedLogClientImpl;
-import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
-import org.apache.distributedlog.client.routing.LocalRoutingService;
-import org.apache.distributedlog.client.routing.RegionsRoutingService;
-import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
-import org.apache.distributedlog.service.stream.StreamManager;
-import org.apache.distributedlog.service.stream.StreamManagerImpl;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration;
-import java.net.SocketAddress;
-import java.net.URI;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-/**
- * Base test case for distributedlog servers.
- */
-public abstract class DistributedLogServerTestCase {
-
-    protected static DistributedLogConfiguration conf =
-            new DistributedLogConfiguration().setLockTimeout(10)
-                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
-    protected static DistributedLogConfiguration noAdHocConf =
-            new DistributedLogConfiguration().setLockTimeout(10).setCreateStreamIfNotExists(false)
-                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
-    protected static DistributedLogCluster dlCluster;
-    protected static DistributedLogCluster noAdHocCluster;
-
-    /**
-     * A distributedlog client wrapper for testing.
-     */
-    protected static class DLClient {
-        public final LocalRoutingService routingService;
-        public DistributedLogClientBuilder dlClientBuilder;
-        public final DistributedLogClientImpl dlClient;
-
-        protected DLClient(String name,
-                           String streamNameRegex,
-                           Optional<String> serverSideRoutingFinagleName) {
-            routingService = LocalRoutingService.newBuilder().build();
-            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
-                        .name(name)
-                        .clientId(ClientId$.MODULE$.apply(name))
-                        .routingService(routingService)
-                        .streamNameRegex(streamNameRegex)
-                        .handshakeWithClientInfo(true)
-                        .clientBuilder(ClientBuilder.get()
-                            .hostConnectionLimit(1)
-                            .connectionTimeout(Duration.fromSeconds(1))
-                            .requestTimeout(Duration.fromSeconds(60)));
-            if (serverSideRoutingFinagleName.isPresent()) {
-                dlClientBuilder =
-                        dlClientBuilder.serverRoutingServiceFinagleNameStr(serverSideRoutingFinagleName.get());
-            }
-            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
-        }
-
-        public void handshake() {
-            dlClient.handshake();
-        }
-
-        public void shutdown() {
-            dlClient.close();
-        }
-    }
-
-    /**
-     * A distributedlog client wrapper that talks to two regions.
-     */
-    protected static class TwoRegionDLClient {
-
-        public final LocalRoutingService localRoutingService;
-        public final LocalRoutingService remoteRoutingService;
-        public final DistributedLogClientBuilder dlClientBuilder;
-        public final DistributedLogClientImpl dlClient;
-
-        protected TwoRegionDLClient(String name, Map<SocketAddress, String> regionMap) {
-            localRoutingService = new LocalRoutingService();
-            remoteRoutingService = new LocalRoutingService();
-            RegionsRoutingService regionsRoutingService =
-                    RegionsRoutingService.of(new DefaultRegionResolver(regionMap),
-                            localRoutingService, remoteRoutingService);
-            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
-                        .name(name)
-                        .clientId(ClientId$.MODULE$.apply(name))
-                        .routingService(regionsRoutingService)
-                        .streamNameRegex(".*")
-                        .handshakeWithClientInfo(true)
-                        .maxRedirects(2)
-                        .clientBuilder(ClientBuilder.get()
-                            .hostConnectionLimit(1)
-                            .connectionTimeout(Duration.fromSeconds(1))
-                            .requestTimeout(Duration.fromSeconds(10)));
-            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
-        }
-
-        public void shutdown() {
-            dlClient.close();
-        }
-    }
-
-    private final boolean clientSideRouting;
-    protected DLServer dlServer;
-    protected DLClient dlClient;
-    protected DLServer noAdHocServer;
-    protected DLClient noAdHocClient;
-
-    public static DistributedLogCluster createCluster(DistributedLogConfiguration conf) throws Exception {
-        return DistributedLogCluster.newBuilder()
-            .numBookies(3)
-            .shouldStartZK(true)
-            .zkServers("127.0.0.1")
-            .shouldStartProxy(false)
-            .dlConf(conf)
-            .bkConf(DLMTestUtil.loadTestBkConf())
-            .build();
-    }
-
-    @BeforeClass
-    public static void setupCluster() throws Exception {
-        dlCluster = createCluster(conf);
-        dlCluster.start();
-    }
-
-    public void setupNoAdHocCluster() throws Exception {
-        noAdHocCluster = createCluster(noAdHocConf);
-        noAdHocCluster.start();
-        noAdHocServer = new DLServer(noAdHocConf, noAdHocCluster.getUri(), 7002, false);
-        Optional<String> serverSideRoutingFinagleName = Optional.absent();
-        if (!clientSideRouting) {
-            serverSideRoutingFinagleName =
-                    Optional.of("inet!" + DLSocketAddress.toString(noAdHocServer.getAddress()));
-        }
-        noAdHocClient = createDistributedLogClient("no-ad-hoc-client", serverSideRoutingFinagleName);
-    }
-
-    public void tearDownNoAdHocCluster() throws Exception {
-        if (null != noAdHocClient) {
-            noAdHocClient.shutdown();
-        }
-        if (null != noAdHocServer) {
-            noAdHocServer.shutdown();
-        }
-    }
-
-    @AfterClass
-    public static void teardownCluster() throws Exception {
-        if (null != dlCluster) {
-            dlCluster.stop();
-        }
-        if (null != noAdHocCluster) {
-            noAdHocCluster.stop();
-        }
-    }
-
-    protected static URI getUri() {
-        return dlCluster.getUri();
-    }
-
-    protected DistributedLogServerTestCase(boolean clientSideRouting) {
-        this.clientSideRouting = clientSideRouting;
-    }
-
-    @Before
-    public void setup() throws Exception {
-        dlServer = createDistributedLogServer(7001);
-        Optional<String> serverSideRoutingFinagleName = Optional.absent();
-        if (!clientSideRouting) {
-            serverSideRoutingFinagleName =
-                    Optional.of("inet!" + DLSocketAddress.toString(dlServer.getAddress()));
-        }
-        dlClient = createDistributedLogClient("test", serverSideRoutingFinagleName);
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != dlClient) {
-            dlClient.shutdown();
-        }
-        if (null != dlServer) {
-            dlServer.shutdown();
-        }
-    }
-
-    protected DLServer createDistributedLogServer(int port) throws Exception {
-        return new DLServer(conf, dlCluster.getUri(), port, false);
-    }
-
-    protected DLServer createDistributedLogServer(DistributedLogConfiguration conf, int port)
-            throws Exception {
-        return new DLServer(conf, dlCluster.getUri(), port, false);
-    }
-
-    protected DLClient createDistributedLogClient(String clientName,
-                                                  Optional<String> serverSideRoutingFinagleName)
-            throws Exception {
-        return createDistributedLogClient(clientName, ".*", serverSideRoutingFinagleName);
-    }
-
-    protected DLClient createDistributedLogClient(String clientName,
-                                                  String streamNameRegex,
-                                                  Optional<String> serverSideRoutingFinagleName)
-            throws Exception {
-        return new DLClient(clientName, streamNameRegex, serverSideRoutingFinagleName);
-    }
-
-    protected TwoRegionDLClient createTwoRegionDLClient(String clientName,
-                                                        Map<SocketAddress, String> regionMap)
-            throws Exception {
-        return new TwoRegionDLClient(clientName, regionMap);
-    }
-
-    protected static void checkStreams(int numExpectedStreams, DLServer dlServer) {
-        StreamManager streamManager = dlServer.dlServer.getKey().getStreamManager();
-        assertEquals(numExpectedStreams, streamManager.numCached());
-        assertEquals(numExpectedStreams, streamManager.numAcquired());
-    }
-
-    protected static void checkStreams(Set<String> streams, DLServer dlServer) {
-        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
-        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
-        Set<String> acquiredStreams = streamManager.getAcquiredStreams().keySet();
-
-        assertEquals(streams.size(), cachedStreams.size());
-        assertEquals(streams.size(), acquiredStreams.size());
-        assertTrue(Sets.difference(streams, cachedStreams).isEmpty());
-        assertTrue(Sets.difference(streams, acquiredStreams).isEmpty());
-    }
-
-    protected static void checkStream(String name, DLClient dlClient, DLServer dlServer,
-                                      int expectedNumProxiesInClient, int expectedClientCacheSize,
-                                      int expectedServerCacheSize, boolean existedInServer, boolean existedInClient) {
-        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
-        assertEquals(expectedNumProxiesInClient, distribution.size());
-
-        if (expectedNumProxiesInClient > 0) {
-            Map.Entry<SocketAddress, Set<String>> localEntry =
-                    distribution.entrySet().iterator().next();
-            assertEquals(dlServer.getAddress(), localEntry.getKey());
-            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
-            assertEquals(existedInClient, localEntry.getValue().contains(name));
-        }
-
-        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
-        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
-        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
-
-        assertEquals(expectedServerCacheSize, cachedStreams.size());
-        assertEquals(existedInServer, cachedStreams.contains(name));
-        assertEquals(expectedServerCacheSize, acquiredStreams.size());
-        assertEquals(existedInServer, acquiredStreams.contains(name));
-    }
-
-    protected static Map<SocketAddress, Set<String>> getStreamOwnershipDistribution(DLClient dlClient) {
-        return dlClient.dlClient.getStreamOwnershipDistribution();
-    }
-
-    protected static Set<String> getAllStreamsFromDistribution(Map<SocketAddress, Set<String>> distribution) {
-        Set<String> allStreams = new HashSet<String>();
-        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
-            allStreams.addAll(entry.getValue());
-        }
-        return allStreams;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
deleted file mode 100644
index 29a3617..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
+++ /dev/null
@@ -1,720 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.Charsets.UTF_8;
-import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-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.base.Optional;
-import org.apache.distributedlog.AsyncLogReader;
-import org.apache.distributedlog.DLMTestUtil;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.DistributedLogManager;
-import org.apache.distributedlog.LogReader;
-import org.apache.distributedlog.LogRecord;
-import org.apache.distributedlog.LogRecordWithDLSN;
-import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
-import org.apache.distributedlog.client.routing.LocalRoutingService;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.LogNotFoundException;
-import org.apache.distributedlog.impl.acl.ZKAccessControl;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.service.stream.StreamManagerImpl;
-import org.apache.distributedlog.thrift.AccessControlEntry;
-import org.apache.distributedlog.thrift.service.BulkWriteResponse;
-import org.apache.distributedlog.thrift.service.HeartbeatOptions;
-import org.apache.distributedlog.thrift.service.StatusCode;
-import org.apache.distributedlog.thrift.service.WriteContext;
-import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.Futures;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link DistributedLogServer}.
- */
-public abstract class TestDistributedLogServerBase extends DistributedLogServerTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogServerBase.class);
-
-    @Rule
-    public TestName testName = new TestName();
-
-    protected TestDistributedLogServerBase(boolean clientSideRouting) {
-        super(clientSideRouting);
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-27}.
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    public void testBasicWrite() throws Exception {
-        String name = "dlserver-basic-write";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        for (long i = 1; i <= 10; i++) {
-            logger.debug("Write entry {} to stream {}.", i, name);
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())));
-        }
-
-        HeartbeatOptions hbOptions = new HeartbeatOptions();
-        hbOptions.setSendHeartBeatToReader(true);
-        // make sure the first log segment of each stream created
-        FutureUtils.result(dlClient.dlClient.heartbeat(name));
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(1);
-        int numRead = 0;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            ++numRead;
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead, i);
-            r = reader.readNext(false);
-        }
-        assertEquals(10, numRead);
-        reader.close();
-        dlm.close();
-    }
-
-    /**
-     * Sanity check to make sure both checksum flag values work.
-     */
-    @Test(timeout = 60000)
-    public void testChecksumFlag() throws Exception {
-        String name = "testChecksumFlag";
-        LocalRoutingService routingService = LocalRoutingService.newBuilder().build();
-        routingService.addHost(name, dlServer.getAddress());
-        DistributedLogClientBuilder dlClientBuilder = DistributedLogClientBuilder.newBuilder()
-            .name(name)
-            .clientId(ClientId$.MODULE$.apply("test"))
-            .routingService(routingService)
-            .handshakeWithClientInfo(true)
-            .clientBuilder(ClientBuilder.get()
-                .hostConnectionLimit(1)
-                .connectionTimeout(Duration.fromSeconds(1))
-                .requestTimeout(Duration.fromSeconds(60)))
-            .checksum(false);
-        DistributedLogClient dlClient = dlClientBuilder.build();
-        Await.result(dlClient.write(name, ByteBuffer.wrap(("1").getBytes())));
-        dlClient.close();
-
-        dlClient = dlClientBuilder.checksum(true).build();
-        Await.result(dlClient.write(name, ByteBuffer.wrap(("2").getBytes())));
-        dlClient.close();
-    }
-
-    private void runSimpleBulkWriteTest(int writeCount) throws Exception {
-        String name = String.format("dlserver-bulk-write-%d", writeCount);
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-
-        logger.debug("Write {} entries to stream {}.", writeCount, name);
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        assertEquals(futures.size(), writeCount);
-        for (Future<DLSN> future : futures) {
-            // No throw == pass.
-            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
-        }
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(1);
-        int numRead = 0;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead + 1, i);
-            ++numRead;
-            r = reader.readNext(false);
-        }
-        assertEquals(writeCount, numRead);
-        reader.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWrite() throws Exception {
-        runSimpleBulkWriteTest(100);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteSingleWrite() throws Exception {
-        runSimpleBulkWriteTest(1);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteEmptyList() throws Exception {
-        String name = String.format("dlserver-bulk-write-%d", 0);
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-
-        assertEquals(0, futures.size());
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteNullArg() throws Exception {
-
-        String name = String.format("dlserver-bulk-write-%s", "null");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
-        writes.add(null);
-
-        try {
-            dlClient.dlClient.writeBulk(name, writes);
-            fail("should not have succeeded");
-        } catch (NullPointerException npe) {
-            // expected
-            logger.info("Expected to catch NullPointException.");
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteEmptyBuffer() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "empty");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
-        writes.add(ByteBuffer.wrap(("").getBytes()));
-        writes.add(ByteBuffer.wrap(("").getBytes()));
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        assertEquals(2, futures.size());
-        for (Future<DLSN> future : futures) {
-            // No throw == pass
-            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
-        }
-    }
-
-    void failDueToWrongException(Exception ex) {
-        logger.info("testBulkWritePartialFailure: ", ex);
-        fail(String.format("failed with wrong exception %s", ex.getClass().getName()));
-    }
-
-    int validateAllFailedAsCancelled(List<Future<DLSN>> futures, int start, int finish) {
-        int failed = 0;
-        for (int i = start; i < finish; i++) {
-            Future<DLSN> future = futures.get(i);
-            try {
-                Await.result(future, Duration.fromSeconds(10));
-                fail("future should have failed!");
-            } catch (DLException cre) {
-                ++failed;
-            } catch (Exception ex) {
-                failDueToWrongException(ex);
-            }
-        }
-        return failed;
-    }
-
-    void validateFailedAsLogRecordTooLong(Future<DLSN> future) {
-        try {
-            Await.result(future, Duration.fromSeconds(10));
-            fail("should have failed");
-        } catch (DLException dle) {
-            assertEquals(StatusCode.TOO_LARGE_RECORD, dle.getCode());
-        } catch (Exception ex) {
-            failDueToWrongException(ex);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWritePartialFailure() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "partial-failure");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        final int writeCount = 100;
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount * 2 + 1);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-        // Too big, will cause partial failure.
-        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
-        writes.add(buf);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-
-        // Count succeeded.
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        int succeeded = 0;
-        for (int i = 0; i < writeCount; i++) {
-            Future<DLSN> future = futures.get(i);
-            try {
-                Await.result(future, Duration.fromSeconds(10));
-                ++succeeded;
-            } catch (Exception ex) {
-                failDueToWrongException(ex);
-            }
-        }
-
-        validateFailedAsLogRecordTooLong(futures.get(writeCount));
-        FutureUtils.result(Futures.collect(futures.subList(writeCount + 1, 2 * writeCount + 1)));
-        assertEquals(writeCount, succeeded);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteTotalFailureFirstWriteFailed() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "first-write-failed");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        final int writeCount = 100;
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
-        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
-        writes.add(buf);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        validateFailedAsLogRecordTooLong(futures.get(0));
-        FutureUtils.result(Futures.collect(futures.subList(1, writeCount + 1)));
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteTotalFailureLostLock() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "lost-lock");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        final int writeCount = 8;
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
-        ByteBuffer buf = ByteBuffer.allocate(8);
-        writes.add(buf);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-        // Warm it up with a write.
-        Await.result(dlClient.dlClient.write(name, ByteBuffer.allocate(8)));
-
-        // Failpoint a lost lock, make sure the failure gets promoted to an operation failure.
-        DistributedLogServiceImpl svcImpl = (DistributedLogServiceImpl) dlServer.dlServer.getLeft();
-        try {
-            FailpointUtils.setFailpoint(
-                FailpointUtils.FailPointName.FP_WriteInternalLostLock,
-                FailpointUtils.FailPointActions.FailPointAction_Default
-            );
-            Future<BulkWriteResponse> futures = svcImpl.writeBulkWithContext(name, writes, new WriteContext());
-            assertEquals(StatusCode.LOCKING_EXCEPTION, Await.result(futures).header.code);
-        } finally {
-            FailpointUtils.removeFailpoint(
-                FailpointUtils.FailPointName.FP_WriteInternalLostLock
-            );
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testHeartbeat() throws Exception {
-        String name = "dlserver-heartbeat";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        for (long i = 1; i <= 10; i++) {
-            logger.debug("Send heartbeat {} to stream {}.", i, name);
-            dlClient.dlClient.check(name).get();
-        }
-
-        logger.debug("Write entry one to stream {}.", name);
-        dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes())).get();
-
-        Thread.sleep(1000);
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
-        int numRead = 0;
-        // eid=0 => control records
-        // other 9 heartbeats will not trigger writing any control records.
-        // eid=1 => user entry
-        long startEntryId = 1;
-        LogRecordWithDLSN r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead + 1, i);
-            assertEquals(r.getDlsn().compareTo(new DLSN(1, startEntryId, 0)), 0);
-            ++numRead;
-            ++startEntryId;
-            r = reader.readNext(false);
-        }
-        assertEquals(1, numRead);
-    }
-
-    @Test(timeout = 60000)
-    public void testFenceWrite() throws Exception {
-        String name = "dlserver-fence-write";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        for (long i = 1; i <= 10; i++) {
-            logger.debug("Write entry {} to stream {}.", i, name);
-            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
-        }
-
-        Thread.sleep(1000);
-
-        logger.info("Fencing stream {}.", name);
-        DLMTestUtil.fenceStream(conf, getUri(), name);
-        logger.info("Fenced stream {}.", name);
-
-        for (long i = 11; i <= 20; i++) {
-            logger.debug("Write entry {} to stream {}.", i, name);
-            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
-        }
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(1);
-        int numRead = 0;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead + 1, i);
-            ++numRead;
-            r = reader.readNext(false);
-        }
-        assertEquals(20, numRead);
-        reader.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testDeleteStream() throws Exception {
-        String name = "dlserver-delete-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        long txid = 101;
-        for (long i = 1; i <= 10; i++) {
-            long curTxId = txid++;
-            logger.debug("Write entry {} to stream {}.", curTxId, name);
-            dlClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-        }
-
-        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
-
-        dlClient.dlClient.delete(name).get();
-
-        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
-
-        Thread.sleep(1000);
-
-        DistributedLogManager dlm101 = DLMTestUtil.createNewDLM(name, conf, getUri());
-        AsyncLogReader reader101 = FutureUtils.result(dlm101.openAsyncLogReader(DLSN.InitialDLSN));
-        try {
-            FutureUtils.result(reader101.readNext());
-            fail("Should fail with LogNotFoundException since the stream is deleted");
-        } catch (LogNotFoundException lnfe) {
-            // expected
-        }
-        FutureUtils.result(reader101.asyncClose());
-        dlm101.close();
-
-        txid = 201;
-        for (long i = 1; i <= 10; i++) {
-            long curTxId = txid++;
-            logger.debug("Write entry {} to stream {}.", curTxId, name);
-            DLSN dlsn = dlClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-        }
-        Thread.sleep(1000);
-
-        DistributedLogManager dlm201 = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader201 = dlm201.getInputStream(1);
-        int numRead = 0;
-        int curTxId = 201;
-        LogRecord r = reader201.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(curTxId++, i);
-            ++numRead;
-            r = reader201.readNext(false);
-        }
-        assertEquals(10, numRead);
-        reader201.close();
-        dlm201.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateStream() throws Exception {
-        try {
-            setupNoAdHocCluster();
-            final String name = "dlserver-create-stream";
-
-            noAdHocClient.routingService.addHost("dlserver-create-stream", noAdHocServer.getAddress());
-            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
-
-            long txid = 101;
-            for (long i = 1; i <= 10; i++) {
-                long curTxId = txid++;
-                logger.debug("Write entry {} to stream {}.", curTxId, name);
-                noAdHocClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-            }
-
-            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-        } finally {
-            tearDownNoAdHocCluster();
-        }
-    }
-
-    /**
-     * This tests that create has touch like behavior in that trying to create the stream twice, simply does nothing.
-     */
-    @Test(timeout = 60000)
-    public void testCreateStreamTwice() throws Exception {
-        try {
-            setupNoAdHocCluster();
-            final String name = "dlserver-create-stream-twice";
-
-            noAdHocClient.routingService.addHost("dlserver-create-stream-twice", noAdHocServer.getAddress());
-            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
-
-            long txid = 101;
-            for (long i = 1; i <= 10; i++) {
-                long curTxId = txid++;
-                logger.debug("Write entry {} to stream {}.", curTxId, name);
-                noAdHocClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-            }
-
-            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-
-            // create again
-            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
-            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-        } finally {
-            tearDownNoAdHocCluster();
-        }
-    }
-
-
-
-    @Test(timeout = 60000)
-    public void testTruncateStream() throws Exception {
-        String name = "dlserver-truncate-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        long txid = 1;
-        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
-        for (int s = 1; s <= 2; s++) {
-            for (long i = 1; i <= 10; i++) {
-                long curTxId = txid++;
-                logger.debug("Write entry {} to stream {}.", curTxId, name);
-                DLSN dlsn = dlClient.dlClient.write(name,
-                        ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-                txid2DLSN.put(curTxId, dlsn);
-            }
-            if (s == 1) {
-                dlClient.dlClient.release(name).get();
-            }
-        }
-
-        DLSN dlsnToDelete = txid2DLSN.get(11L);
-        dlClient.dlClient.truncate(name, dlsnToDelete).get();
-
-        DistributedLogManager readDLM = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = readDLM.getInputStream(1);
-        int numRead = 0;
-        int curTxId = 11;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(curTxId++, i);
-            ++numRead;
-            r = reader.readNext(false);
-        }
-        assertEquals(10, numRead);
-        reader.close();
-        readDLM.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testRequestDenied() throws Exception {
-        String name = "request-denied";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        AccessControlEntry ace = new AccessControlEntry();
-        ace.setDenyWrite(true);
-        ZooKeeperClient zkc = TestZooKeeperClientBuilder
-                .newBuilder()
-                .uri(getUri())
-                .connectionTimeoutMs(60000)
-                .sessionTimeoutMs(60000)
-                .build();
-        DistributedLogNamespace dlNamespace = dlServer.dlServer.getLeft().getDistributedLogNamespace();
-        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
-        String zkPath = getUri().getPath() + "/" + bkdlConfig.getACLRootPath() + "/" + name;
-        ZKAccessControl accessControl = new ZKAccessControl(ace, zkPath);
-        accessControl.create(zkc);
-
-        AccessControlManager acm = dlNamespace.createAccessControlManager();
-        while (acm.allowWrite(name)) {
-            Thread.sleep(100);
-        }
-
-        try {
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-            fail("Should fail with request denied exception");
-        } catch (DLException dle) {
-            assertEquals(StatusCode.REQUEST_DENIED, dle.getCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testNoneStreamNameRegex() throws Exception {
-        String streamNamePrefix = "none-stream-name-regex-";
-        int numStreams = 5;
-        Set<String> streams = new HashSet<String>();
-
-        for (int i = 0; i < numStreams; i++) {
-            streams.add(streamNamePrefix + i);
-        }
-        testStreamNameRegex(streams, ".*", streams);
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamNameRegex() throws Exception {
-        String streamNamePrefix = "stream-name-regex-";
-        int numStreams = 5;
-        Set<String> streams = new HashSet<String>();
-        Set<String> expectedStreams = new HashSet<String>();
-        String streamNameRegex = streamNamePrefix + "1";
-
-        for (int i = 0; i < numStreams; i++) {
-            streams.add(streamNamePrefix + i);
-        }
-        expectedStreams.add(streamNamePrefix + "1");
-
-        testStreamNameRegex(streams, streamNameRegex, expectedStreams);
-    }
-
-    private void testStreamNameRegex(Set<String> streams, String streamNameRegex,
-                                     Set<String> expectedStreams)
-            throws Exception {
-        for (String streamName : streams) {
-            dlClient.routingService.addHost(streamName, dlServer.getAddress());
-            Await.result(dlClient.dlClient.write(streamName,
-                    ByteBuffer.wrap(streamName.getBytes(UTF_8))));
-        }
-
-        DLClient client = createDistributedLogClient(
-                "test-stream-name-regex",
-                streamNameRegex,
-                Optional.<String>absent());
-        try {
-            client.routingService.addHost("unknown", dlServer.getAddress());
-            client.handshake();
-            Map<SocketAddress, Set<String>> distribution =
-                    client.dlClient.getStreamOwnershipDistribution();
-            assertEquals(1, distribution.size());
-            Set<String> cachedStreams = distribution.values().iterator().next();
-            assertNotNull(cachedStreams);
-            assertEquals(expectedStreams.size(), cachedStreams.size());
-
-            for (String streamName : cachedStreams) {
-                assertTrue(expectedStreams.contains(streamName));
-            }
-        } finally {
-            client.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testReleaseStream() throws Exception {
-        String name = "dlserver-release-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
-
-        // release the stream
-        Await.result(dlClient.dlClient.release(name));
-        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
-    }
-
-    protected void checkStream(int expectedNumProxiesInClient, int expectedClientCacheSize, int expectedServerCacheSize,
-                             String name, SocketAddress owner, boolean existedInServer, boolean existedInClient) {
-        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
-        assertEquals(expectedNumProxiesInClient, distribution.size());
-
-        if (expectedNumProxiesInClient > 0) {
-            Map.Entry<SocketAddress, Set<String>> localEntry =
-                    distribution.entrySet().iterator().next();
-            assertEquals(owner, localEntry.getKey());
-            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
-            assertEquals(existedInClient, localEntry.getValue().contains(name));
-        }
-
-
-        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
-        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
-        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
-
-        assertEquals(expectedServerCacheSize, cachedStreams.size());
-        assertEquals(existedInServer, cachedStreams.contains(name));
-        assertEquals(expectedServerCacheSize, acquiredStreams.size());
-        assertEquals(existedInServer, acquiredStreams.contains(name));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java
deleted file mode 100644
index c7ae960..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.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 org.apache.distributedlog.service;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.fail;
-
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import org.junit.Test;
-
-/**
- * Test the server with client side routing.
- */
-public class TestDistributedLogServerClientRouting extends TestDistributedLogServerBase {
-
-    public TestDistributedLogServerClientRouting() {
-        super(true);
-    }
-
-    @Test(timeout = 60000)
-    public void testAcceptNewStream() throws Exception {
-        String name = "dlserver-accept-new-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-        dlClient.routingService.setAllowRetrySameHost(false);
-
-        Await.result(dlClient.dlClient.setAcceptNewStream(false));
-
-        try {
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-            fail("Should fail because the proxy couldn't accept new stream");
-        } catch (NoBrokersAvailableException nbae) {
-            // expected
-        }
-        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
-
-        Await.result(dlServer.dlServer.getLeft().setAcceptNewStream(true));
-        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java
deleted file mode 100644
index 12416a3..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.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 org.apache.distributedlog.service;
-
-/**
- * Test the server with client side routing.
- */
-public class TestDistributedLogServerServerRouting extends TestDistributedLogServerBase {
-
-    public TestDistributedLogServerServerRouting() {
-        super(false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
deleted file mode 100644
index e5d75c2..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
+++ /dev/null
@@ -1,833 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Lists;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.TestDistributedLogBase;
-import org.apache.distributedlog.acl.DefaultAccessControlManager;
-import org.apache.distributedlog.client.routing.LocalRoutingService;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.exceptions.StreamUnavailableException;
-import org.apache.distributedlog.service.config.NullStreamConfigProvider;
-import org.apache.distributedlog.service.config.ServerConfiguration;
-import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
-import org.apache.distributedlog.service.stream.Stream;
-import org.apache.distributedlog.service.stream.StreamImpl;
-import org.apache.distributedlog.service.stream.StreamImpl.StreamStatus;
-import org.apache.distributedlog.service.stream.StreamManagerImpl;
-import org.apache.distributedlog.service.stream.WriteOp;
-import org.apache.distributedlog.service.streamset.DelimiterStreamPartitionConverter;
-import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.distributedlog.thrift.service.HeartbeatOptions;
-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.ConfUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.ProtocolUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.ConfigurationException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for DistributedLog Service.
- */
-public class TestDistributedLogService extends TestDistributedLogBase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogService.class);
-
-    @Rule
-    public TestName testName = new TestName();
-
-    private ServerConfiguration serverConf;
-    private DistributedLogConfiguration dlConf;
-    private URI uri;
-    private final CountDownLatch latch = new CountDownLatch(1);
-    private DistributedLogServiceImpl service;
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        dlConf = new DistributedLogConfiguration();
-        dlConf.addConfiguration(conf);
-        dlConf.setLockTimeout(0)
-                .setOutputBufferSize(0)
-                .setPeriodicFlushFrequencyMilliSeconds(10)
-                .setSchedulerShutdownTimeoutMs(100);
-        serverConf = newLocalServerConf();
-        uri = createDLMURI("/" + testName.getMethodName());
-        ensureURICreated(uri);
-        service = createService(serverConf, dlConf, latch);
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        if (null != service) {
-            service.shutdown();
-        }
-        super.teardown();
-    }
-
-    private DistributedLogConfiguration newLocalConf() {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(dlConf);
-        return confLocal;
-    }
-
-    private ServerConfiguration newLocalServerConf() {
-        ServerConfiguration serverConf = new ServerConfiguration();
-        serverConf.loadConf(dlConf);
-        serverConf.setServerThreads(1);
-        return serverConf;
-    }
-
-    private DistributedLogServiceImpl createService(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf) throws Exception {
-        return createService(serverConf, dlConf, new CountDownLatch(1));
-    }
-
-    private DistributedLogServiceImpl createService(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf,
-            CountDownLatch latch) throws Exception {
-        // Build the stream partition converter
-        StreamPartitionConverter converter;
-        try {
-            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
-        } catch (ConfigurationException e) {
-            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
-                    IdentityStreamPartitionConverter.class.getName());
-            converter = new IdentityStreamPartitionConverter();
-        }
-        return new DistributedLogServiceImpl(
-            serverConf,
-            dlConf,
-            ConfUtils.getConstDynConf(dlConf),
-            new NullStreamConfigProvider(),
-            uri,
-            converter,
-            new LocalRoutingService(),
-            NullStatsLogger.INSTANCE,
-            NullStatsLogger.INSTANCE,
-            latch,
-            new EqualLoadAppraiser());
-    }
-
-    private StreamImpl createUnstartedStream(DistributedLogServiceImpl service,
-                                             String name) throws Exception {
-        StreamImpl stream = (StreamImpl) service.newStream(name);
-        stream.initialize();
-        return stream;
-    }
-
-    private ByteBuffer createRecord(long txid) {
-        return ByteBuffer.wrap(("record-" + txid).getBytes(UTF_8));
-    }
-
-    private WriteOp createWriteOp(DistributedLogServiceImpl service,
-                                  String streamName,
-                                  long txid) {
-        ByteBuffer data = createRecord(txid);
-        return service.newWriteOp(streamName, data, null);
-    }
-
-    @Test(timeout = 60000)
-    public void testAcquireStreams() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s0 = createUnstartedStream(service, streamName);
-        ServerConfiguration serverConf1 = new ServerConfiguration();
-        serverConf1.addConfiguration(serverConf);
-        serverConf1.setServerPort(9999);
-        DistributedLogServiceImpl service1 = createService(serverConf1, dlConf);
-        StreamImpl s1 = createUnstartedStream(service1, streamName);
-
-        // create write ops
-        WriteOp op0 = createWriteOp(service, streamName, 0L);
-        s0.submit(op0);
-
-        WriteOp op1 = createWriteOp(service1, streamName, 1L);
-        s1.submit(op1);
-
-        // check pending size
-        assertEquals("Write Op 0 should be pending in service 0",
-                1, s0.numPendingOps());
-        assertEquals("Write Op 1 should be pending in service 1",
-                1, s1.numPendingOps());
-
-        // start acquiring s0
-        s0.start();
-        WriteResponse wr0 = Await.result(op0.result());
-        assertEquals("Op 0 should succeed",
-                StatusCode.SUCCESS, wr0.getHeader().getCode());
-        assertEquals("Service 0 should acquire stream",
-                StreamStatus.INITIALIZED, s0.getStatus());
-        assertNotNull(s0.getManager());
-        assertNotNull(s0.getWriter());
-        assertNull(s0.getLastException());
-
-        // start acquiring s1
-        s1.start();
-        WriteResponse wr1 = Await.result(op1.result());
-        assertEquals("Op 1 should fail",
-                StatusCode.FOUND, wr1.getHeader().getCode());
-        // the stream will be set to ERROR and then be closed.
-        assertTrue("Service 1 should be in unavailable state",
-                StreamStatus.isUnavailable(s1.getStatus()));
-        assertNotNull(s1.getManager());
-        assertNull(s1.getWriter());
-        assertNotNull(s1.getLastException());
-        assertTrue(s1.getLastException() instanceof OwnershipAcquireFailedException);
-
-        service1.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testAcquireStreamsWhenExceedMaxCachedPartitions() throws Exception {
-        String streamName = testName.getMethodName() + "_0000";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(dlConf);
-        confLocal.setMaxCachedPartitionsPerProxy(1);
-
-        ServerConfiguration serverConfLocal = new ServerConfiguration();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
-
-        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
-        Stream stream = serviceLocal.getLogWriter(streamName);
-
-        // stream is cached
-        assertNotNull(stream);
-        assertEquals(1, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op0 = createWriteOp(service, streamName, 0L);
-        stream.submit(op0);
-        WriteResponse wr0 = Await.result(op0.result());
-        assertEquals("Op 0 should succeed",
-                StatusCode.SUCCESS, wr0.getHeader().getCode());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-
-        // should fail to acquire another partition
-        try {
-            serviceLocal.getLogWriter(testName.getMethodName() + "_0001");
-            fail("Should fail to acquire new streams");
-        } catch (StreamUnavailableException sue) {
-            // expected
-        }
-        assertEquals(1, serviceLocal.getStreamManager().numCached());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-
-        // should be able to acquire partitions from other streams
-        String anotherStreamName = testName.getMethodName() + "-another_0001";
-        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
-        assertNotNull(anotherStream);
-        assertEquals(2, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
-        anotherStream.submit(op1);
-        WriteResponse wr1 = Await.result(op1.result());
-        assertEquals("Op 1 should succeed",
-                StatusCode.SUCCESS, wr1.getHeader().getCode());
-        assertEquals(2, serviceLocal.getStreamManager().numAcquired());
-    }
-
-    @Test(timeout = 60000)
-    public void testAcquireStreamsWhenExceedMaxAcquiredPartitions() throws Exception {
-        String streamName = testName.getMethodName() + "_0000";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(dlConf);
-        confLocal.setMaxCachedPartitionsPerProxy(-1);
-        confLocal.setMaxAcquiredPartitionsPerProxy(1);
-
-        ServerConfiguration serverConfLocal = new ServerConfiguration();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
-
-        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
-        Stream stream = serviceLocal.getLogWriter(streamName);
-
-        // stream is cached
-        assertNotNull(stream);
-        assertEquals(1, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op0 = createWriteOp(service, streamName, 0L);
-        stream.submit(op0);
-        WriteResponse wr0 = Await.result(op0.result());
-        assertEquals("Op 0 should succeed",
-                StatusCode.SUCCESS, wr0.getHeader().getCode());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-
-        // should be able to cache partitions from same stream
-        String anotherStreamName = testName.getMethodName() + "_0001";
-        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
-        assertNotNull(anotherStream);
-        assertEquals(2, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
-        anotherStream.submit(op1);
-        WriteResponse wr1 = Await.result(op1.result());
-        assertEquals("Op 1 should fail",
-                StatusCode.STREAM_UNAVAILABLE, wr1.getHeader().getCode());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseShouldErrorOutPendingOps() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s = createUnstartedStream(service, streamName);
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
-        for (int i = 0; i < numWrites; i++) {
-            WriteOp op = createWriteOp(service, streamName, i);
-            s.submit(op);
-            futureList.add(op.result());
-        }
-        assertEquals(numWrites, s.numPendingOps());
-        Await.result(s.requestClose("close stream"));
-        assertEquals("Stream " + streamName + " is set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-        for (int i = 0; i < numWrites; i++) {
-            Future<WriteResponse> future = futureList.get(i);
-            WriteResponse wr = Await.result(future);
-            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
-                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseTwice() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s = createUnstartedStream(service, streamName);
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
-        for (int i = 0; i < numWrites; i++) {
-            WriteOp op = createWriteOp(service, streamName, i);
-            s.submit(op);
-            futureList.add(op.result());
-        }
-        assertEquals(numWrites, s.numPendingOps());
-
-        Future<Void> closeFuture0 = s.requestClose("close 0");
-        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
-                StreamStatus.CLOSING == s.getStatus()
-                    || StreamStatus.CLOSED == s.getStatus());
-        Future<Void> closeFuture1 = s.requestClose("close 1");
-        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
-                StreamStatus.CLOSING == s.getStatus()
-                    || StreamStatus.CLOSED == s.getStatus());
-
-        Await.result(closeFuture0);
-        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-        Await.result(closeFuture1);
-        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-
-        for (int i = 0; i < numWrites; i++) {
-            Future<WriteResponse> future = futureList.get(i);
-            WriteResponse wr = Await.result(future);
-            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
-                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testFailRequestsDuringClosing() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s = createUnstartedStream(service, streamName);
-
-        Future<Void> closeFuture = s.requestClose("close");
-        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
-                StreamStatus.CLOSING == s.getStatus()
-                    || StreamStatus.CLOSED == s.getStatus());
-        WriteOp op1 = createWriteOp(service, streamName, 0L);
-        s.submit(op1);
-        WriteResponse response1 = Await.result(op1.result());
-        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closing",
-                StatusCode.STREAM_UNAVAILABLE, response1.getHeader().getCode());
-
-        Await.result(closeFuture);
-        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-        WriteOp op2 = createWriteOp(service, streamName, 1L);
-        s.submit(op2);
-        WriteResponse response2 = Await.result(op2.result());
-        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closed",
-                StatusCode.STREAM_UNAVAILABLE, response2.getHeader().getCode());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceTimeout() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-        ServerConfiguration serverConfLocal = newLocalServerConf();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.setServiceTimeoutMs(200)
-                .setStreamProbationTimeoutMs(100);
-        String streamName = testName.getMethodName();
-        // create a new service with 200ms timeout
-        DistributedLogServiceImpl localService = createService(serverConfLocal, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
-        for (int i = 0; i < numWrites; i++) {
-            futureList.add(localService.write(streamName, createRecord(i)));
-        }
-
-        assertTrue("Stream " + streamName + " should be cached",
-                streamManager.getCachedStreams().containsKey(streamName));
-
-        StreamImpl s = (StreamImpl) streamManager.getCachedStreams().get(streamName);
-        // the stream should be set CLOSING
-        while (StreamStatus.CLOSING != s.getStatus()
-            && StreamStatus.CLOSED != s.getStatus()) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-        assertNotNull("Writer should be initialized", s.getWriter());
-        assertNull("No exception should be thrown", s.getLastException());
-        Future<Void> closeFuture = s.getCloseFuture();
-        Await.result(closeFuture);
-        for (int i = 0; i < numWrites; i++) {
-            assertTrue("Write should not fail before closing",
-                    futureList.get(i).isDefined());
-            WriteResponse response = Await.result(futureList.get(i));
-            assertTrue("Op should fail with " + StatusCode.WRITE_CANCELLED_EXCEPTION,
-                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
-                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
-                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
-        }
-
-        while (streamManager.getCachedStreams().containsKey(streamName)) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        assertFalse("Stream should be removed from cache",
-                streamManager.getCachedStreams().containsKey(streamName));
-        assertFalse("Stream should be removed from acquired cache",
-                streamManager.getAcquiredStreams().containsKey(streamName));
-
-        localService.shutdown();
-    }
-
-    private DistributedLogServiceImpl createConfiguredLocalService() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-        return createService(serverConf, confLocal);
-    }
-
-    private ByteBuffer getTestDataBuffer() {
-        return ByteBuffer.wrap("test-data".getBytes());
-    }
-
-    @Test(timeout = 60000)
-    public void testNonDurableWrite() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0)
-                .setDurableWriteEnabled(false);
-        ServerConfiguration serverConfLocal = new ServerConfiguration();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.enableDurableWrite(false);
-        serverConfLocal.setServiceTimeoutMs(Integer.MAX_VALUE)
-                .setStreamProbationTimeoutMs(Integer.MAX_VALUE);
-        String streamName = testName.getMethodName();
-        DistributedLogServiceImpl localService =
-                createService(serverConfLocal, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>();
-        for (int i = 0; i < numWrites; i++) {
-            futureList.add(localService.write(streamName, createRecord(i)));
-        }
-        assertTrue("Stream " + streamName + " should be cached",
-                streamManager.getCachedStreams().containsKey(streamName));
-        List<WriteResponse> resultList = FutureUtils.result(Future.collect(futureList));
-        for (WriteResponse wr : resultList) {
-            assertEquals(DLSN.InvalidDLSN, DLSN.deserialize(wr.getDlsn()));
-        }
-
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpNoChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext();
-        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testTruncateOpNoChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext();
-        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpNoChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext();
-        HeartbeatOptions option = new HeartbeatOptions();
-        option.setSendHeartBeatToReader(true);
-
-        // hearbeat to acquire the stream and then release the stream
-        Future<WriteResponse> result = localService.heartbeatWithOptions("test", ctx, option);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        result = localService.release("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-
-        // heartbeat to acquire the stream and then delete the stream
-        result = localService.heartbeatWithOptions("test", ctx, option);
-        resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        result = localService.delete("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-
-        // shutdown the local service
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpChecksumBadChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(999);
-        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpChecksumBadStream() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(
-            ProtocolUtils.writeOpCRC32("test", getTestDataBuffer().array()));
-        Future<WriteResponse> result = localService.writeWithContext("test1", getTestDataBuffer(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpChecksumBadData() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        ByteBuffer buffer = getTestDataBuffer();
-        WriteContext ctx = new WriteContext().setCrc32(
-            ProtocolUtils.writeOpCRC32("test", buffer.array()));
-
-        // Overwrite 1 byte to corrupt data.
-        buffer.put(1, (byte) 0xab);
-        Future<WriteResponse> result = localService.writeWithContext("test", buffer, ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpChecksumBadChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(999);
-        Future<WriteResponse> result = localService.heartbeat("test", ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        result = localService.release("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        result = localService.delete("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testTruncateOpChecksumBadChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(999);
-        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    private WriteOp getWriteOp(String name, SettableFeature disabledFeature, Long checksum) {
-        return new WriteOp(name,
-            ByteBuffer.wrap("test".getBytes()),
-            new NullStatsLogger(),
-            new NullStatsLogger(),
-            new IdentityStreamPartitionConverter(),
-            new ServerConfiguration(),
-            (byte) 0,
-            checksum,
-            false,
-            disabledFeature,
-            DefaultAccessControlManager.INSTANCE);
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpBadChecksumWithChecksumDisabled() throws Exception {
-        String streamName = testName.getMethodName();
-
-        SettableFeature disabledFeature = new SettableFeature("", 0);
-
-        WriteOp writeOp0 = getWriteOp(streamName, disabledFeature, 919191L);
-        WriteOp writeOp1 = getWriteOp(streamName, disabledFeature, 919191L);
-
-        try {
-            writeOp0.preExecute();
-            fail("should have thrown");
-        } catch (Exception ex) {
-        }
-
-        disabledFeature.set(1);
-        writeOp1.preExecute();
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpGoodChecksumWithChecksumDisabled() throws Exception {
-        String streamName = testName.getMethodName();
-
-        SettableFeature disabledFeature = new SettableFeature("", 1);
-        WriteOp writeOp0 = getWriteOp(
-            streamName,
-            disabledFeature,
-            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
-        WriteOp writeOp1 = getWriteOp(
-            streamName,
-            disabledFeature,
-            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
-
-        writeOp0.preExecute();
-        disabledFeature.set(0);
-        writeOp1.preExecute();
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseStreamsShouldFlush() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-
-        String streamNamePrefix = testName.getMethodName();
-        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numStreams = 10;
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList =
-                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
-        for (int i = 0; i < numStreams; i++) {
-            String streamName = streamNamePrefix + "-" + i;
-            HeartbeatOptions hbOptions = new HeartbeatOptions();
-            hbOptions.setSendHeartBeatToReader(true);
-            // make sure the first log segment of each stream created
-            FutureUtils.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
-            for (int j = 0; j < numWrites; j++) {
-                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
-            }
-        }
-
-        assertEquals("There should be " + numStreams + " streams in cache",
-                numStreams, streamManager.getCachedStreams().size());
-        while (streamManager.getAcquiredStreams().size() < numStreams) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        Future<List<Void>> closeResult = localService.closeStreams();
-        List<Void> closedStreams = Await.result(closeResult);
-        assertEquals("There should be " + numStreams + " streams closed",
-                numStreams, closedStreams.size());
-        // all writes should be flushed
-        for (Future<WriteResponse> future : futureList) {
-            WriteResponse response = Await.result(future);
-            assertTrue("Op should succeed or be rejected : " + response.getHeader().getCode(),
-                    StatusCode.SUCCESS == response.getHeader().getCode()
-                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
-                        || StatusCode.STREAM_UNAVAILABLE == response.getHeader().getCode());
-        }
-        assertTrue("There should be no streams in the cache",
-                streamManager.getCachedStreams().isEmpty());
-        assertTrue("There should be no streams in the acquired cache",
-                streamManager.getAcquiredStreams().isEmpty());
-
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseStreamsShouldAbort() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-
-        String streamNamePrefix = testName.getMethodName();
-        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numStreams = 10;
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList =
-                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
-        for (int i = 0; i < numStreams; i++) {
-            String streamName = streamNamePrefix + "-" + i;
-            HeartbeatOptions hbOptions = new HeartbeatOptions();
-            hbOptions.setSendHeartBeatToReader(true);
-            // make sure the first log segment of each stream created
-            FutureUtils.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
-            for (int j = 0; j < numWrites; j++) {
-                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
-            }
-        }
-
-        assertEquals("There should be " + numStreams + " streams in cache",
-                numStreams, streamManager.getCachedStreams().size());
-        while (streamManager.getAcquiredStreams().size() < numStreams) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        for (Stream s : streamManager.getAcquiredStreams().values()) {
-            StreamImpl stream = (StreamImpl) s;
-            stream.setStatus(StreamStatus.ERROR);
-        }
-
-        Future<List<Void>> closeResult = localService.closeStreams();
-        List<Void> closedStreams = Await.result(closeResult);
-        assertEquals("There should be " + numStreams + " streams closed",
-                numStreams, closedStreams.size());
-        // all writes should be flushed
-        for (Future<WriteResponse> future : futureList) {
-            WriteResponse response = Await.result(future);
-            assertTrue("Op should fail with " + StatusCode.BK_TRANSMIT_ERROR + " or be rejected : "
-                    + response.getHeader().getCode(),
-                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
-                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
-                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
-        }
-        // acquired streams should all been removed after we close them
-        assertTrue("There should be no streams in the acquired cache",
-            streamManager.getAcquiredStreams().isEmpty());
-        localService.shutdown();
-        // cached streams wouldn't be removed immediately after streams are closed
-        // but they should be removed after we shutdown the service
-        assertTrue("There should be no streams in the cache after shutting down the service",
-            streamManager.getCachedStreams().isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testShutdown() throws Exception {
-        service.shutdown();
-        StreamManagerImpl streamManager = (StreamManagerImpl) service.getStreamManager();
-        WriteResponse response =
-                Await.result(service.write(testName.getMethodName(), createRecord(0L)));
-        assertEquals("Write should fail with " + StatusCode.SERVICE_UNAVAILABLE,
-                StatusCode.SERVICE_UNAVAILABLE, response.getHeader().getCode());
-        assertTrue("There should be no streams created after shutdown",
-                streamManager.getCachedStreams().isEmpty());
-        assertTrue("There should be no streams acquired after shutdown",
-                streamManager.getAcquiredStreams().isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetOwner() throws Exception {
-        ((LocalRoutingService) service.getRoutingService())
-                .addHost("stream-0", service.getServiceAddress().getSocketAddress())
-                .setAllowRetrySameHost(false);
-
-        service.startPlacementPolicy();
-
-        WriteResponse response = FutureUtils.result(service.getOwner("stream-1", new WriteContext()));
-        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
-        assertEquals(service.getServiceAddress().toString(),
-                response.getHeader().getLocation());
-
-        // service cache "stream-2"
-        StreamImpl stream = (StreamImpl) service.getStreamManager().getOrCreateStream("stream-2", false);
-        // create write ops to stream-2 to make service acquire the stream
-        WriteOp op = createWriteOp(service, "stream-2", 0L);
-        stream.submit(op);
-        stream.start();
-        WriteResponse wr = Await.result(op.result());
-        assertEquals("Op should succeed",
-                StatusCode.SUCCESS, wr.getHeader().getCode());
-        assertEquals("Service should acquire stream",
-                StreamStatus.INITIALIZED, stream.getStatus());
-        assertNotNull(stream.getManager());
-        assertNotNull(stream.getWriter());
-        assertNull(stream.getLastException());
-
-        // the stream is acquired
-        response = FutureUtils.result(service.getOwner("stream-2", new WriteContext()));
-        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
-        assertEquals(service.getServiceAddress().toString(),
-                response.getHeader().getLocation());
-    }
-
-}


[19/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-protocol/pom.xml b/distributedlog-proxy-protocol/pom.xml
new file mode 100644
index 0000000..0f6a85f
--- /dev/null
+++ b/distributedlog-proxy-protocol/pom.xml
@@ -0,0 +1,130 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.distributedlog</groupId>
+    <artifactId>distributedlog</artifactId>
+    <version>0.5.0-incubating-SNAPSHOT</version>
+  </parent>
+  <artifactId>distributedlog-proxy-protocol</artifactId>
+  <name>Apache DistributedLog :: Proxy Protocol</name>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-protocol</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libthrift</artifactId>
+      <version>${libthrift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>scrooge-core_2.11</artifactId>
+      <version>${scrooge.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-core_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-thrift_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>com.twitter</groupId>
+        <artifactId>scrooge-maven-plugin</artifactId>
+        <version>${scrooge-maven-plugin.version}</version>
+        <configuration>
+          <language>java</language>
+          <thriftOpts>
+            <thriftOpt>--finagle</thriftOpt>
+          </thriftOpts>
+        </configuration>
+        <executions>
+          <execution>
+            <id>thrift-sources</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/src/main/resources/findbugsExclude.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven-checkstyle-plugin.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>com.puppycrawl.tools</groupId>
+            <artifactId>checkstyle</artifactId>
+            <version>${puppycrawl.checkstyle.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.distributedlog</groupId>
+            <artifactId>distributedlog-build-tools</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <consoleOutput>true</consoleOutput>
+          <failOnViolation>true</failOnViolation>
+          <includeResources>false</includeResources>
+          <includeTestSourceDirectory>true</includeTestSourceDirectory>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>test-compile</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/ProtocolUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/ProtocolUtils.java b/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/ProtocolUtils.java
new file mode 100644
index 0000000..1f91968
--- /dev/null
+++ b/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/ProtocolUtils.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.protocol.util;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import org.apache.distributedlog.DLSN;
+import java.util.zip.CRC32;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+
+/**
+ * With CRC embedded in the application, we have to keep track of per api crc. Ideally this
+ * would be done by thrift.
+ */
+public class ProtocolUtils {
+
+    // For request payload checksum
+    private static final ThreadLocal<CRC32> requestCRC = new ThreadLocal<CRC32>() {
+        @Override
+        protected CRC32 initialValue() {
+            return new CRC32();
+        }
+    };
+
+    /**
+     * Generate crc32 for WriteOp.
+     */
+    public static Long writeOpCRC32(String stream, byte[] payload) {
+        CRC32 crc = requestCRC.get();
+        try {
+            crc.update(stream.getBytes(UTF_8));
+            crc.update(payload);
+            return crc.getValue();
+        } finally {
+            crc.reset();
+        }
+    }
+
+    /**
+     * Generate crc32 for TruncateOp.
+     */
+    public static Long truncateOpCRC32(String stream, DLSN dlsn) {
+        CRC32 crc = requestCRC.get();
+        try {
+            crc.update(stream.getBytes(UTF_8));
+            crc.update(dlsn.serializeBytes());
+            return crc.getValue();
+        } finally {
+            crc.reset();
+        }
+    }
+
+    /**
+     * Generate crc32 for any op which only passes a stream name.
+     */
+    public static Long streamOpCRC32(String stream) {
+        CRC32 crc = requestCRC.get();
+        try {
+            crc.update(stream.getBytes(UTF_8));
+            return crc.getValue();
+        } finally {
+            crc.reset();
+        }
+    }
+
+    public static DLException exception(ResponseHeader response) {
+        String errMsg;
+        switch (response.getCode()) {
+            case FOUND:
+                if (response.isSetErrMsg()) {
+                    errMsg = response.getErrMsg();
+                } else {
+                    errMsg = "Request is redirected to " + response.getLocation();
+                }
+                return new OwnershipAcquireFailedException(errMsg, response.getLocation());
+            case SUCCESS:
+                throw new IllegalArgumentException("Can't instantiate an exception for success response.");
+            default:
+                if (response.isSetErrMsg()) {
+                    errMsg = response.getErrMsg();
+                } else {
+                    errMsg = response.getCode().name();
+                }
+                return new DLException(response.getCode().getValue(), errMsg);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/package-info.java b/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/package-info.java
new file mode 100644
index 0000000..311f6c1
--- /dev/null
+++ b/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/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.
+ */
+/**
+ * defines the protocol related utilities.
+ */
+package org.apache.distributedlog.protocol.util;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-protocol/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-protocol/src/main/resources/findbugsExclude.xml b/distributedlog-proxy-protocol/src/main/resources/findbugsExclude.xml
new file mode 100644
index 0000000..05ee085
--- /dev/null
+++ b/distributedlog-proxy-protocol/src/main/resources/findbugsExclude.xml
@@ -0,0 +1,23 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+//-->
+<FindBugsFilter>
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
+  </Match>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-protocol/src/main/thrift/service.thrift
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-protocol/src/main/thrift/service.thrift b/distributedlog-proxy-protocol/src/main/thrift/service.thrift
new file mode 100644
index 0000000..45e1449
--- /dev/null
+++ b/distributedlog-proxy-protocol/src/main/thrift/service.thrift
@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+namespace java org.apache.distributedlog.thrift.service
+
+/* Response stats codes */
+enum StatusCode {
+    /* 2xx: action requested by the client was received, understood, accepted and processed successfully. */
+
+    /* standard response for successful requests. */
+    SUCCESS = 200,
+
+    /* 3xx: client must take additional action to complete the request. */
+
+    /* client closed. */
+    CLIENT_CLOSED = 301,
+    /* found the stream in a different server, a redirection is required by client. */
+    FOUND = 302,
+
+    /* 4xx: client seems to have erred. */
+
+    /* request is denied for some reason */
+    REQUEST_DENIED = 403,
+    /* request record too large */
+    TOO_LARGE_RECORD = 413,
+
+    /* 5xx: server failed to fulfill an apparently valid request. */
+
+    /* Generic error message, given when no more specific message is suitable. */
+    INTERNAL_SERVER_ERROR = 500,
+    /* Not implemented */
+    NOT_IMPLEMENTED = 501,
+    /* Already Closed Exception */
+    ALREADY_CLOSED = 502,
+    /* Service is currently unavailable (because it is overloaded or down for maintenance). */
+    SERVICE_UNAVAILABLE = 503,
+    /* Locking exception */
+    LOCKING_EXCEPTION = 504,
+    /* ZooKeeper Errors */
+    ZOOKEEPER_ERROR = 505,
+    /* Metadata exception */
+    METADATA_EXCEPTION = 506,
+    /* BK Transmit Error */
+    BK_TRANSMIT_ERROR = 507,
+    /* Flush timeout */
+    FLUSH_TIMEOUT = 508,
+    /* Log empty */
+    LOG_EMPTY = 509,
+    /* Log not found */
+    LOG_NOT_FOUND = 510,
+    /* Truncated Transactions */
+    TRUNCATED_TRANSACTION = 511,
+    /* End of Stream */
+    END_OF_STREAM = 512,
+    /* Transaction Id Out of Order */
+    TRANSACTION_OUT_OF_ORDER = 513,
+    /* Write exception */
+    WRITE_EXCEPTION = 514,
+    /* Stream Unavailable */
+    STREAM_UNAVAILABLE = 515,
+    /* Write cancelled exception */
+    WRITE_CANCELLED_EXCEPTION = 516,
+    /* over-capacity/backpressure */
+    OVER_CAPACITY = 517,
+    /** stream exists but is not ready (recovering etc.).
+        the difference between NOT_READY and UNAVAILABLE is that UNAVAILABLE
+        indicates the stream is no longer owned by the proxy and we should
+        redirect. NOT_READY indicates the stream exist at the proxy but isn't
+        eady for writes. */
+    STREAM_NOT_READY = 518,
+    /* Region Unavailable */
+    REGION_UNAVAILABLE = 519,
+    /* Invalid Enveloped Entry */
+    INVALID_ENVELOPED_ENTRY = 520,
+    /* Unsupported metadata version */
+    UNSUPPORTED_METADATA_VERSION = 521,
+    /* Log Already Exists */
+    LOG_EXISTS = 522,
+    /* Checksum failed on the request */
+    CHECKSUM_FAILED = 523,
+    /* Overcapacity: too many streams */
+    TOO_MANY_STREAMS = 524,
+    /* Log Segment Not Found */
+    LOG_SEGMENT_NOT_FOUND = 525,
+    /* End of Log Segment */
+    END_OF_LOG_SEGMENT = 526,
+    /* Log Segment Is Truncated */
+    LOG_SEGMENT_IS_TRUNCATED = 527,
+
+    /* 6xx: unexpected */
+
+    UNEXPECTED = 600,
+    INTERRUPTED = 601,
+    INVALID_STREAM_NAME = 602,
+    ILLEGAL_STATE = 603,
+
+    /* 10xx: reader exceptions */
+
+    RETRYABLE_READ = 1000,
+    LOG_READ_ERROR = 1001,
+    /* Read cancelled exception */
+    READ_CANCELLED_EXCEPTION = 1002,
+}
+
+/* Response Header */
+struct ResponseHeader {
+    1: required StatusCode code;
+    2: optional string errMsg;
+    3: optional string location;
+}
+
+/* Write Response */
+struct WriteResponse {
+    1: required ResponseHeader header;
+    2: optional string dlsn;
+}
+
+/* Bulk write response */
+struct BulkWriteResponse {
+    1: required ResponseHeader header;
+    2: optional list<WriteResponse> writeResponses;
+}
+
+/* Write Context */
+struct WriteContext {
+    1: optional set<string> triedHosts;
+    2: optional i64 crc32;
+    3: optional bool isRecordSet;
+}
+
+/* HeartBeat Options */
+struct HeartbeatOptions {
+    1: optional bool sendHeartBeatToReader;
+}
+
+/* Server Status */
+enum ServerStatus {
+    /* service is writing and accepting new streams */
+    WRITE_AND_ACCEPT    = 100,
+    /* service is only writing to old streams, not accepting new streams */
+    WRITE_ONLY          = 200,
+    /* service is shutting down, will not write */
+    DOWN                = 300,
+}
+
+/* Server Info */
+struct ServerInfo {
+    1: optional map<string, string> ownerships;
+    2: optional ServerStatus serverStatus;
+}
+
+/* Client Info */
+struct ClientInfo {
+    1: optional string streamNameRegex;
+    2: optional bool getOwnerships;
+}
+
+service DistributedLogService {
+
+    /* Deprecated */
+    ServerInfo handshake();
+
+    ServerInfo handshakeWithClientInfo(ClientInfo clientInfo);
+
+    /* Deprecated */
+    WriteResponse heartbeat(string stream, WriteContext ctx);
+
+    WriteResponse heartbeatWithOptions(string stream, WriteContext ctx, HeartbeatOptions options);
+
+    /* Deprecated */
+    WriteResponse write(string stream, binary data);
+
+    WriteResponse writeWithContext(string stream, binary data, WriteContext ctx);
+
+    BulkWriteResponse writeBulkWithContext(string stream, list<binary> data, WriteContext ctx);
+
+    WriteResponse truncate(string stream, string dlsn, WriteContext ctx);
+
+    WriteResponse release(string stream, WriteContext ctx);
+
+    WriteResponse create(string stream, WriteContext ctx);
+
+    WriteResponse delete(string stream, WriteContext ctx);
+
+    WriteResponse getOwner(string stream, WriteContext ctx);
+
+    /* Admin Methods */
+    void setAcceptNewStream(bool enabled);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/bin/bundle
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/bin/bundle b/distributedlog-proxy-server/bin/bundle
new file mode 100755
index 0000000..1b1904e
--- /dev/null
+++ b/distributedlog-proxy-server/bin/bundle
@@ -0,0 +1,28 @@
+#!/bin/bash
+#
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+set -e
+
+BASEDIR=$(dirname "$0")
+DLOG_ROOT="${BASEDIR}/../.."
+
+cd "${DLOG_ROOT}"
+
+bash scripts/bundle service

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/bin/common.sh
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/bin/common.sh b/distributedlog-proxy-server/bin/common.sh
new file mode 100755
index 0000000..2b13157
--- /dev/null
+++ b/distributedlog-proxy-server/bin/common.sh
@@ -0,0 +1,124 @@
+#!/usr/bin/env bash
+#
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+if [ $(uname) == "Linux" ]; then
+  # check if net.ipv6.bindv6only is set to 1
+  bindv6only=$(/sbin/sysctl -n net.ipv6.bindv6only 2> /dev/null)
+  if [ -n "${bindv6only}" ] && [ "${bindv6only}" -eq "1" ]; then
+    echo "Error: \"net.ipv6.bindv6only\" is set to 1 - Java networking could be broken"
+    echo "For more info (the following page also applies to DistributedLog): http://wiki.apache.org/hadoop/HadoopIPv6"
+    exit 1
+  fi
+fi
+
+# See the following page for extensive details on setting
+# up the JVM to accept JMX remote management:
+# http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
+# by default we allow local JMX connections
+if [ -z "${JMXLOCALONLY}" ]; then
+  JMXLOCALONLY=false
+fi
+
+if [ -z "${JMXDISABLE}" ]; then
+  echo "JMX enabled by default" >&2
+  # for some reason these two options are necessary on jdk6 on Ubuntu
+  # accord to the docs they are not necessary, but otw jconsole cannot
+  # do a local attach
+  JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=${JMXLOCALONLY}"
+else
+  echo "JMX disabled by user request" >&2
+fi
+
+echo "DLOG_HOME => ${DLOG_HOME}"
+DEFAULT_LOG_CONF="${DLOG_HOME}/conf/log4j.properties"
+
+[ -f "${DLOG_HOME}/conf/dlogenv.sh" ] && source "${DLOG_HOME}/conf/dlogenv.sh"
+
+# exclude tests jar
+RELEASE_JAR=$(ls ${DLOG_HOME}/distributedlog-*.jar 2> /dev/null | grep -v 'tests\|javadoc\|sources' | tail -1)
+if [ $? == 0 ]; then
+  DLOG_JAR="${RELEASE_JAR}"
+fi
+
+# exclude tests jar
+BUILT_JAR=$(ls ${DLOG_HOME}/target/distributedlog-*.jar 2> /dev/null | grep -v 'tests\|javadoc\|sources' | tail -1)
+
+if [ -e "${BUILD_JAR}" ] && [ -e "${DLOG_JAR}" ]; then
+  echo "\nCouldn't find dlog jar.";
+  echo "Make sure you've run 'mvn package'\n";
+  exit 1;
+elif [ -e "${BUILT_JAR}" ]; then
+  DLOG_JAR="${BUILT_JAR}"
+fi
+
+add_maven_deps_to_classpath() {
+  MVN="mvn"
+  if [ -n "${MAVEN_HOME}" ]; then
+    MVN="${MAVEN_HOME}/bin/mvn"
+  fi
+
+  # Need to generate classpath from maven pom. This is costly so generate it
+  # and cache it. Save the file into our target dir so a mvn clean will get
+  # clean it up and force us create a new one.
+  f="${PWD}/${DLOG_HOME}/target/cached_classpath.txt"
+  if [ ! -f "${f}" ]; then
+    "${MVN}" -f "${DLOG_HOME}/pom.xml" dependency:build-classpath -Dmdep.outputFile="${f}" &> /dev/null
+  fi
+  DLOG_CLASSPATH="${CLASSPATH}":$(cat "${f}")
+}
+
+if [ -d "${DLOG_HOME}/lib" ]; then
+  for i in ${DLOG_HOME}/lib/*.jar; do
+    DLOG_CLASSPATH="${DLOG_CLASSPATH}:${i}"
+  done
+else
+  add_maven_deps_to_classpath
+fi
+
+# if no args specified, exit
+if [ $# = 0 ]; then
+  exit 1
+fi
+
+if [ -z "${DLOG_LOG_CONF}" ]; then
+  DLOG_LOG_CONF="${DEFAULT_LOG_CONF}"
+fi
+
+DLOG_CLASSPATH="${DLOG_JAR}:${DLOG_CLASSPATH}:${DLOG_EXTRA_CLASSPATH}"
+if [ -n "${DLOG_LOG_CONF}" ]; then
+  DLOG_CLASSPATH="$(dirname ${DLOG_LOG_CONF}):${DLOG_CLASSPATH}"
+  OPTS="${OPTS} -Dlog4j.configuration=$(basename ${DLOG_LOG_CONF})"
+fi
+OPTS="-cp ${DLOG_CLASSPATH} ${OPTS} ${DLOG_EXTRA_OPTS}"
+
+OPTS="${OPTS} ${DLOG_EXTRA_OPTS}"
+
+# Disable ipv6 as it can cause issues
+OPTS="${OPTS} -Djava.net.preferIPv4Stack=true"
+
+# log directory & file
+DLOG_ROOT_LOGGER=${DLOG_ROOT_LOGGER:-"INFO,R"}
+DLOG_LOG_DIR=${DLOG_LOG_DIR:-"$DLOG_HOME/logs"}
+DLOG_LOG_FILE=${DLOG_LOG_FILE:-"dlog.log"}
+
+#Configure log configuration system properties
+OPTS="$OPTS -Ddlog.root.logger=${DLOG_ROOT_LOGGER}"
+OPTS="$OPTS -Ddlog.log.dir=${DLOG_LOG_DIR}"
+OPTS="$OPTS -Ddlog.log.file=${DLOG_LOG_FILE}"

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/bin/dlog
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/bin/dlog b/distributedlog-proxy-server/bin/dlog
new file mode 100755
index 0000000..99cad40
--- /dev/null
+++ b/distributedlog-proxy-server/bin/dlog
@@ -0,0 +1,88 @@
+#!/usr/bin/env bash
+#
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+set -e
+
+BASEDIR=$(dirname "$0")
+DLOG_HOME="${BASEDIR}/.."
+
+usage() {
+  cat <<EOF
+Usage: runner <command>
+where command is one of:
+  local               Run distributedlog sandbox
+  example             Run distributedlog example
+  tool                Run distributedlog tool
+  proxy_tool          Run distributedlog proxy tool to interact with proxies
+  balancer            Run distributedlog balancer
+  admin               Run distributedlog admin tool
+  zkshell             Run zookeeper shell
+  bkshell             Run bookkeeper shell
+  help                This help message
+
+or command is the full name of a class with a defined main() method.
+
+Environment variables:
+  DLOG_LOG_CONF        Log4j configuration file (default $DEFAULT_LOG_CONF)
+  DLOG_EXTRA_OPTS      Extra options to be passed to the jvm
+  DLOG_EXTRA_CLASSPATH Add extra paths to the dlog classpath
+
+These variable can also be set in conf/dlogenv.sh
+EOF
+}
+
+source "${DLOG_HOME}"/bin/common.sh
+
+# get arguments
+COMMAND=$1
+shift
+
+case "${COMMAND}" in
+  local)
+    exec java $OPTS $JMX_ARGS org.apache.distributedlog.LocalDLMEmulator $@
+    ;;
+  tool)
+    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.tools.DistributedLogTool $@
+    ;;
+  proxy_tool)
+    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.service.tools.ProxyTool $@
+    ;;
+  balancer)
+    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.service.balancer.BalancerTool $@
+    ;;
+  admin)
+    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.admin.DistributedLogAdmin $@
+    ;;
+  zkshell)
+    exec java $OPTS org.apache.zookeeper.ZooKeeperMain -server $@
+    ;;
+  bkshell)
+    ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}"
+    exec java $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF $@
+    ;;
+  help)
+    usage
+    ;;
+  *)
+    exec java $OPTS $COMMAND $@
+    ;;
+esac
+
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/bin/dlog-daemon.sh
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/bin/dlog-daemon.sh b/distributedlog-proxy-server/bin/dlog-daemon.sh
new file mode 100755
index 0000000..fdd87df
--- /dev/null
+++ b/distributedlog-proxy-server/bin/dlog-daemon.sh
@@ -0,0 +1,200 @@
+#!/usr/bin/env bash
+#
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+usage() {
+  cat <<EOF
+Usage: dlog-daemon.sh (start|stop) <service> <args...>
+where service is one of:
+  zookeeper                   Run the zookeeper server
+  bookie                      Run the bookie server
+  bookie-rereplicator         Run the bookie rereplicator
+  writeproxy                  Run the write proxy server
+  writeproxy-monitor          Run the write proxy monitor
+
+where argument is one of:
+  -force (accepted only with stop service): Decides whether to stop the process forcefully if not stopped by normal shutdown
+EOF
+}
+
+BINDIR=$(dirname "$0")
+DL_HOME=$(cd $BINDIR/..;pwd)
+
+if [ -f "${DL_HOME}/conf/dlogenv.sh" ]; then
+ . "${DL_HOME}/conf/dlogenv.sh"
+fi
+
+SERVICE_PORT=${SERVICE_PORT:-"0"}
+SERVICE_ARGS=""
+
+# DLOG logging configuration
+DLOG_LOG_DIR=${DLOG_LOG_DIR:-"$DL_HOME/logs"}
+DLOG_ROOT_LOGGER=${DLOG_ROOT_LOGGER:-'INFO,R'}
+
+# Process Control Parameters
+DLOG_STOP_TIMEOUT=${DLOG_STOP_TIMEOUT:-30}
+DLOG_PID_DIR=${DLOG_PID_DIR:-$DL_HOME/pids}
+
+if [ $# -lt 2 ]; then
+  echo "Error: not enough arguments provided."
+  usage
+  exit 1
+fi
+
+command=$1
+shift
+
+service=$1
+shift
+
+service_class=$service
+case $service in
+  (zookeeper)
+    service_class="org.apache.zookeeper.server.quorum.QuorumPeerMain"
+    DLOG_ROOT_LOGGER=${ZK_ROOT_LOGGER:-'INFO,R'}
+    ;;
+  (bookie)
+    service_class="org.apache.bookkeeper.proto.BookieServer"
+    DLOG_ROOT_LOGGER=${BK_ROOT_LOGGER:-'INFO,R'}
+    ;;
+  (bookie-rereplicator)
+    service_class="org.apache.bookkeeper.replication.AutoRecoveryMain"
+    DLOG_ROOT_LOGGER=${BK_ROOT_LOGGER:-'INFO,R'}
+    ;;
+  (writeproxy)
+    service_class="org.apache.distributedlog.service.DistributedLogServerApp"
+    DLOG_ROOT_LOGGER=${WP_ROOT_LOGGER:-'INFO,R'}
+    WP_CONF_FILE=${WP_CONF_FILE:-"$DL_HOME/conf/write_proxy.conf"}
+    WP_SERVICE_PORT=${WP_SERVICE_PORT:-'4181'}
+    WP_STATS_PORT=${WP_STATS_PORT:-'9000'}
+    WP_STATS_PROVIDER=${WP_STATS_PROVIDER:-'org.apache.bookkeeper.stats.CodahaleMetricsServletProvider'}
+    WP_SHARD_ID=${WP_SHARD_ID:-'0'}
+    WP_NAMESPACE=${WP_NAMESPACE:-'distributedlog://127.0.0.1:2181/messaging/distributedlog/mynamespace'}
+    SERVICE_PORT=${WP_SERVICE_PORT}
+    SERVICE_ARGS="--conf ${WP_CONF_FILE} --uri ${WP_NAMESPACE} --shard-id ${WP_SHARD_ID} --port ${WP_SERVICE_PORT} --stats-port ${WP_STATS_PORT} --stats-provider ${WP_STATS_PROVIDER} --announce --thriftmux"
+    DLOG_EXTRA_OPTS="${DLOG_EXTRA_OPTS} -DcodahaleStatsHttpPort=${WP_STATS_PORT} -Dserver_port=${WP_SERVICE_PORT} -Dserver_shard=${WP_SHARD_ID}"
+    ;;
+  (writeproxy-monitor)
+    ;;
+  (*)
+    echo "Error: unknown service name $service"
+    usage
+    exit 1
+    ;;
+esac
+
+echo "doing $command $service ..."
+
+export DLOG_LOG_DIR=$DLOG_LOG_DIR
+export DLOG_ROOT_LOGGER=$DLOG_ROOT_LOGGER
+export DLOG_LOG_FILE=dlog-$service-$HOSTNAME-$SERVICE_PORT.log
+export DLOG_EXTRA_OPTS=$DLOG_EXTRA_OPTS
+
+pid=$DLOG_PID_DIR/dlog-$service-$SERVICE_PORT.pid
+out=$DLOG_LOG_DIR/dlog-$service-$HOSTNAME-$SERVICE_PORT.out
+logfile=$DLOG_LOG_DIR/$DLOG_LOG_FILE
+
+rotate_out_log () {
+  log=$1;
+  num=5;
+  if [ -n "$2" ]; then
+   num=$2
+  fi
+  if [ -f "$log" ]; then # rotate logs
+    while [ $num -gt 1 ]; do
+      prev=`expr $num - 1`
+      [ -f "$log.$prev" ] && mv "$log.$prev" "$log.$num"
+      num=$prev
+    done
+    mv "$log" "$log.$num";
+  fi
+}
+
+mkdir -p "$DLOG_LOG_DIR"
+mkdir -p "$DLOG_PID_DIR"
+
+case $command in
+  (start)
+    if [ -f $pid ]; then
+      if kill -0 $(cat $pid) > /dev/null 2>&1; then
+        echo $service running as process $(cat $pid).  Stop it first.
+        exit 1
+      fi
+    fi
+
+    rotate_out_log $out
+    echo starting $service, logging to $logfile
+    dlog=$DL_HOME/bin/dlog
+    nohup $dlog $service_class ${SERVICE_ARGS} "$@" > "$out" 2>&1 < /dev/null &
+    echo $! > $pid
+    sleep 1; head $out
+    sleep 2;
+    if ! ps -p $! > /dev/null ; then
+      exit 1
+    fi
+    ;;
+
+  (stop)
+    if [ -f $pid ]; then
+      TARGET_PID=$(cat $pid)
+      if kill -0 $TARGET_PID > /dev/null 2>&1; then
+        echo stopping $service
+        kill $TARGET_PID
+
+        count=0
+        location=$DLOG_LOG_DIR
+        while ps -p $TARGET_PID > /dev/null; do
+          echo "Shutdown is in progress... Please wait..."
+          sleep 1
+          count=$(expr $count + 1)
+          if [ "$count" = "$DLOG_STOP_TIMEOUT" ]; then
+            break
+          fi
+        done
+        if [ "$count" != "$DLOG_STOP_TIMEOUT" ]; then
+          echo "Shutdown completed."
+        fi
+        if kill -0 $TARGET_PID > /dev/null 2>&1; then
+          fileName=$location/$service.out
+          $JAVA_HOME/bin/jstack $TARGET_PID > $fileName
+          echo Thread dumps are taken for analysis at $fileName
+          if [ "$1" == "-force" ]; then
+            echo forcefully stopping $service
+            kill -9 $TARGET_PID >/dev/null 2>&1
+            echo Successfully stopped the process
+          else
+            echo "WARNNING : $service is not stopped completely."
+            exit 1
+          fi
+        fi
+      else
+        echo no $service to stop
+      fi
+      rm $pid
+    else
+      echo no $service to stop
+    fi
+    ;;
+
+  (*)
+    usage
+    echo $supportedargs
+    exit 1
+    ;;
+esac

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/bin/dlog-env.sh
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/bin/dlog-env.sh b/distributedlog-proxy-server/bin/dlog-env.sh
new file mode 100644
index 0000000..50a1960
--- /dev/null
+++ b/distributedlog-proxy-server/bin/dlog-env.sh
@@ -0,0 +1,35 @@
+#!/usr/bin/env bash
+#
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+# we need the DLog URI to be set
+if [[ -z "${DISTRIBUTEDLOG_URI}" ]]; then
+  echo "Environment variable DISTRIBUTEDLOG_URI is not set."
+  exit 1
+fi
+
+# add the jars from current dir to the class path (should be distributedlog-service)
+for i in ./*.jar; do
+  CLASSPATH="$i:${CLASSPATH}"
+done
+
+# add all the jar from lib/ to the class path
+for i in ./lib/*.jar; do
+  CLASSPATH="$i:${CLASSPATH}"
+done

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/bin/dlog-start.sh
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/bin/dlog-start.sh b/distributedlog-proxy-server/bin/dlog-start.sh
new file mode 100755
index 0000000..c9c2192
--- /dev/null
+++ b/distributedlog-proxy-server/bin/dlog-start.sh
@@ -0,0 +1,34 @@
+#!/bin/bash
+#
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+DLOGBIN="${BASH_SOURCE-$0}"
+DLOGBIN="$(dirname "${DLOGBIN}")"
+DLOGBINDIR="$(cd "${DLOGBIN}"; pwd)"
+
+. "${DLOGBINDIR}"/dlog-env.sh
+
+java -cp "${CLASSPATH}" \
+     -Dlog4j.configuration=conf/log4j.properties \
+     -DstatsHttpPort=9000 -DstatsExport=true \
+     -Dserver_shard=0 \
+     org.apache.distributedlog.service.DistributedLogServerApp \
+     --port 8000 \
+     --uri "${DISTRIBUTEDLOG_URI}" \
+     --conf conf/distributedlog.conf

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/bin/zk-server-start.sh
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/bin/zk-server-start.sh b/distributedlog-proxy-server/bin/zk-server-start.sh
new file mode 100644
index 0000000..f222756
--- /dev/null
+++ b/distributedlog-proxy-server/bin/zk-server-start.sh
@@ -0,0 +1,44 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+# 
+#    http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+if [ $# -lt 1 ];
+then
+    echo "USAGE: $0 [-daemon] zookeeper.properties"
+    exit 1
+fi
+BASE_DIR=$(dirname $0)
+
+if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then
+    export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$BASE_DIR/../config/log4j.properties"
+fi
+
+if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then
+    export KAFKA_HEAP_OPTS="-Xmx512M -Xms512M"
+fi
+
+EXTRA_ARGS="-name zookeeper -loggc"
+
+COMMAND=$1
+case $COMMAND in
+  -daemon)
+     EXTRA_ARGS="-daemon "$EXTRA_ARGS
+     shift
+     ;;
+ *)
+     ;;
+esac
+
+exec $BASE_DIR/kafka-run-class.sh $EXTRA_ARGS org.apache.zookeeper.server.quorum.QuorumPeerMain "$@"

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/conf/bookie.conf.template
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/conf/bookie.conf.template b/distributedlog-proxy-server/conf/bookie.conf.template
new file mode 100644
index 0000000..5ca89d0
--- /dev/null
+++ b/distributedlog-proxy-server/conf/bookie.conf.template
@@ -0,0 +1,183 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+## Bookie settings
+
+# Port that bookie server listen on
+bookiePort=3181
+
+# TODO: change the journal directory
+# Directory Bookkeeper outputs its write ahead log
+journalDirectory=/tmp/data/bk/journal
+
+# TODO: change the ledgers directory
+# Directory Bookkeeper outputs ledger snapshots
+ledgerDirectories=/tmp/data/bk/ledgers
+
+# TODO: change the index directory
+# Directory in which index files will be stored.
+indexDirectories=/tmp/data/bk/ledgers
+
+# Ledger Manager Class
+# What kind of ledger manager is used to manage how ledgers are stored, managed
+# and garbage collected. Try to read 'BookKeeper Internals' for detail info.
+ledgerManagerType=hierarchical
+
+# Root zookeeper path to store ledger metadata
+# This parameter is used by zookeeper-based ledger manager as a root znode to
+# store all ledgers.
+zkLedgersRootPath=/messaging/bookkeeper/ledgers
+
+# Max file size of entry logger, in bytes
+# A new entry log file will be created when the old one reaches the file size limitation
+logSizeLimit=1073741823
+
+# Max file size of journal file, in mega bytes
+# A new journal file will be created when the old one reaches the file size limitation
+#
+journalMaxSizeMB=2048
+
+# Max number of old journal file to kept
+# Keep a number of old journal files would help data recovery in specia case
+#
+journalMaxBackups=5
+
+# How long the interval to trigger next garbage collection, in milliseconds
+# Since garbage collection is running in background, too frequent gc
+# will heart performance. It is better to give a higher number of gc
+# interval if there is enough disk capacity.
+# gc per 1 hour (aligning with most DL rolling interval)
+gcInitialWaitTime=600000
+gcWaitTime=3600000
+# do minor compaction per 2 hours
+minorCompactionInterval=7200
+minorCompactionThreshold=0.2
+# disable major compaction
+majorCompactionInterval=0
+# reduce major compaction threshold to a low value to prevent bad force compaction behavior
+majorCompactionThreshold=0.3
+# Compaction Rate & Max Outstanding
+compactionRate=10737418
+compactionMaxOutstandingRequests=10737418
+
+# How long the interval to flush ledger index pages to disk, in milliseconds
+# Flushing index files will introduce much random disk I/O.
+# If separating journal dir and ledger dirs each on different devices,
+# flushing would not affect performance. But if putting journal dir
+# and ledger dirs on same device, performance degrade significantly
+# on too frequent flushing. You can consider increment flush interval
+# to get better performance, but you need to pay more time on bookie
+# server restart after failure.
+#
+flushInterval=1000
+
+# Interval to watch whether bookie is dead or not, in milliseconds
+#
+# bookieDeathWatchInterval=1000
+
+## zookeeper client settings
+
+# A list of one of more servers on which zookeeper is running.
+# The server list can be comma separated values, for example:
+# zkServers=zk1:2181,zk2:2181,zk3:2181
+zkServers=localhost:2181
+
+# ZooKeeper client session timeout in milliseconds
+# Bookie server will exit if it received SESSION_EXPIRED because it
+# was partitioned off from ZooKeeper for more than the session timeout
+# JVM garbage collection, disk I/O will cause SESSION_EXPIRED.
+# Increment this value could help avoiding this issue
+zkTimeout=30000
+
+## NIO Server settings
+
+# This settings is used to enabled/disabled Nagle's algorithm, which is a means of
+# improving the efficiency of TCP/IP networks by reducing the number of packets
+# that need to be sent over the network.
+# If you are sending many small messages, such that more than one can fit in
+# a single IP packet, setting server.tcpnodelay to false to enable Nagle algorithm
+# can provide better performance.
+# Default value is true.
+#
+serverTcpNoDelay=true
+
+## ledger cache settings
+
+# Max number of ledger index files could be opened in bookie server
+# If number of ledger index files reaches this limitation, bookie
+# server started to swap some ledgers from memory to disk.
+# Too frequent swap will affect performance. You can tune this number
+# to gain performance according your requirements.
+openFileLimit=20000
+
+# Size of a index page in ledger cache, in bytes
+# A larger index page can improve performance writing page to disk,
+# which is efficent when you have small number of ledgers and these
+# ledgers have similar number of entries.
+# If you have large number of ledgers and each ledger has fewer entries,
+# smaller index page would improve memory usage.
+pageSize=8192
+
+# How many index pages provided in ledger cache
+# If number of index pages reaches this limitation, bookie server
+# starts to swap some ledgers from memory to disk. You can increment
+# this value when you found swap became more frequent. But make sure
+# pageLimit*pageSize should not more than JVM max memory limitation,
+# otherwise you would got OutOfMemoryException.
+# In general, incrementing pageLimit, using smaller index page would
+# gain bettern performance in lager number of ledgers with fewer entries case
+# If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute
+# the limitation of number of index pages.
+pageLimit=131072
+
+#If all ledger directories configured are full, then support only read requests for clients.
+#If "readOnlyModeEnabled=true" then on all ledger disks full, bookie will be converted
+#to read-only mode and serve only read requests. Otherwise the bookie will be shutdown.
+readOnlyModeEnabled=true
+
+# Bookie Journal Settings
+writeBufferSizeBytes=262144
+journalFlushWhenQueueEmpty=false
+journalRemoveFromPageCache=true
+journalAdaptiveGroupWrites=true
+journalMaxGroupWaitMSec=4
+journalBufferedEntriesThreshold=180
+journalBufferedWritesThreshold=131072
+journalMaxGroupedEntriesToCommit=200
+journalPreAllocSizeMB=4
+
+# Sorted Ledger Storage Settings
+sortedLedgerStorageEnabled=true
+skipListSizeLimit=67108864
+skipListArenaChunkSize=2097152
+skipListArenaMaxAllocSize=131072
+fileInfoCacheInitialCapacity=10000
+fileInfoMaxIdleTime=3600
+
+# Bookie Threads Settings (NOTE: change this to align the cpu cores)
+numAddWorkerThreads=4
+numJournalCallbackThreads=4
+numReadWorkerThreads=4
+numLongPollWorkerThreads=4
+
+# stats
+statsProviderClass=org.apache.bookkeeper.stats.CodahaleMetricsServletProvider
+# Exporting codahale stats
+codahaleStatsHttpPort=9001
+useHostNameAsBookieID=true
+allowLoopback=true

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/conf/distributedlog.conf
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/conf/distributedlog.conf b/distributedlog-proxy-server/conf/distributedlog.conf
new file mode 100644
index 0000000..dac71ac
--- /dev/null
+++ b/distributedlog-proxy-server/conf/distributedlog.conf
@@ -0,0 +1,125 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+########################
+# ZooKeeper Client Settings
+########################
+
+# zookeeper settings
+zkSessionTimeoutSeconds=30
+zkNumRetries=0
+zkRetryStartBackoffMillis=100
+zkRetryMaxBackoffMillis=200
+# bkc zookeeper settings
+bkcZKSessionTimeoutSeconds=60
+bkcZKNumRetries=20
+bkcZKRetryStartBackoffMillis=100
+bkcZKRetryMaxBackoffMillis=200
+
+########################
+# BookKeeper Client Settings
+########################
+
+# bookkeeper client timeouts
+bkcWriteTimeoutSeconds=10
+bkcReadTimeoutSeconds=1
+bkcNumWorkerThreads=16
+# bkcNumIOThreads=16
+bkc.numChannelsPerBookie=1
+bkc.enableTaskExecutionStats=true
+bkc.connectTimeoutMillis=1000
+bkc.enablePerHostStats=true
+
+########################
+# DL Settings
+########################
+
+# lock timeout
+lockTimeoutSeconds=0
+# dl worker threads
+numWorkerThreads=16
+
+### Recovery Related Settings
+
+# recover log segments in background
+recoverLogSegmentsInBackground=true
+# disable max id in proxy
+maxIdSanityCheck=true
+# use allocator pool for proxy
+enableLedgerAllocatorPool=false
+# ledger allocator pool size
+ledgerAllocatorPoolCoreSize=20
+# check stream exists or not
+createStreamIfNotExists=true
+# encode dc id in version
+encodeDCIDInVersion=true
+# logSegmentNameVersion
+logSegmentNameVersion=1
+
+### Write Performance Related Settings
+
+# ensemble size
+ensemble-size=3
+write-quorum-size=3
+ack-quorum-size=2
+bkc.ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
+bkc.delayEnsembleChange=true
+
+# sync settings
+# buffer size is large because when we rewrite we perform a very large write to persist
+# all queue state at once (up to max queue memory size, ex. 16MB). the write will be
+# throttled if it takes too long, which can hurt performance, so important to optimize
+# for this case.
+output-buffer-size=512000
+enableImmediateFlush=false
+periodicFlushFrequencyMilliSeconds=6
+logFlushTimeoutSeconds=120
+
+### Ledger Rolling Related Settings
+
+# retention policy
+retention-size=0
+# rolling ledgers (disable time rolling/enable size rolling)
+rolling-interval=0
+
+# max logsegment bytes=2GB
+# much larger than max journal size, effectively never roll and let drpc do it
+maxLogSegmentBytes=2147483648
+
+# rolling concurrency
+logSegmentRollingConcurrency=1
+# disable sanityCheckDelete
+sanityCheckDelete=false
+ledgerAllocatorPoolName=drpc-alloc-pool
+
+### Readahead settings
+
+enableReadAhead=true
+ReadAheadBatchSize=10
+ReadAheadMaxEntries=100
+ReadAheadWaitTime=10
+
+### Rate limit
+
+rpsSoftWriteLimit=1
+rpsHardWriteLimit=5
+rpsHardServiceLimit=15
+
+### Config
+
+dynamicConfigReloadIntervalSec=5

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/conf/dlogenv.sh
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/conf/dlogenv.sh b/distributedlog-proxy-server/conf/dlogenv.sh
new file mode 100644
index 0000000..345e60f
--- /dev/null
+++ b/distributedlog-proxy-server/conf/dlogenv.sh
@@ -0,0 +1,75 @@
+#!/bin/sh
+#
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+##################
+# General
+##################
+
+# Log4j configuration file
+# DLOG_LOG_CONF=
+
+# Extra options to be passed to the jvm
+# DLOG_EXTRA_OPTS=
+
+# Add extra paths to the dlog classpath
+# DLOG_EXTRA_CLASSPATH=
+
+# Configure the root logger
+# DLOG_ROOT_LOGGER=
+
+# Configure the log dir
+# DLOG_LOG_DIR=
+
+# Configure the log file
+# DLOG_LOG_FILE=
+
+#################
+# ZooKeeper
+#################
+
+# Configure zookeeper root logger
+# ZK_ROOT_LOGGER=
+
+#################
+# Bookie
+#################
+
+# Configure bookie root logger
+# BK_ROOT_LOGGER=
+
+#################
+# Write Proxy
+#################
+
+# Configure write proxy root logger
+# WP_ROOT_LOGGER=
+
+# write proxy configuration file
+# WP_CONF_FILE=${DL_HOME}/conf/write_proxy.conf
+
+# port and stats port
+# WP_SERVICE_PORT=4181
+# WP_STATS_PORT=9000
+
+# shard id
+# WP_SHARD_ID=0
+
+# write proxy namespace
+# WP_NAMESPACE=distributedlog://127.0.0.1:2181/messaging/distributedlog/mynamespace

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/conf/log4j.properties b/distributedlog-proxy-server/conf/log4j.properties
new file mode 100644
index 0000000..73b4cfa
--- /dev/null
+++ b/distributedlog-proxy-server/conf/log4j.properties
@@ -0,0 +1,60 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT 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 Logging Configuration
+#
+
+# Default values
+dlog.root.logger=INFO, R
+dlog.log.dir=logs
+dlog.log.file=dlog.log
+
+log4j.rootLogger=${dlog.root.logger}
+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.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.bookkeeper.util.SafeRunnable=false
+
+log4j.appender.Executors=org.apache.log4j.RollingFileAppender
+log4j.appender.Executors.Threshold=INFO
+log4j.appender.Executors.File=${dlog.log.dir}/executors.log
+log4j.appender.Executors.MaxFileSize=20MB
+log4j.appender.Executors.MaxBackupIndex=5
+log4j.appender.Executors.layout=org.apache.log4j.PatternLayout
+log4j.appender.Executors.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+log4j.appender.R=org.apache.log4j.RollingFileAppender
+log4j.appender.R.Threshold=INFO
+log4j.appender.R.File=${dlog.log.dir}/${dlog.log.file}
+log4j.appender.R.MaxFileSize=20MB
+log4j.appender.R.MaxBackupIndex=50
+log4j.appender.R.layout=org.apache.log4j.PatternLayout
+log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+log4j.appender.stderr=org.apache.log4j.ConsoleAppender
+log4j.appender.stderr.Target=System.err
+log4j.appender.stderr.Threshold=INFO
+log4j.appender.stderr.layout=org.apache.log4j.PatternLayout
+log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/conf/write_proxy.conf
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/conf/write_proxy.conf b/distributedlog-proxy-server/conf/write_proxy.conf
new file mode 100644
index 0000000..7f5351a
--- /dev/null
+++ b/distributedlog-proxy-server/conf/write_proxy.conf
@@ -0,0 +1,143 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+########################
+# ZooKeeper Client Settings
+########################
+
+# zookeeper settings
+zkSessionTimeoutSeconds=1
+zkNumRetries=0
+zkRetryStartBackoffMillis=100
+zkRetryMaxBackoffMillis=200
+# bkc zookeeper settings
+bkcZKSessionTimeoutSeconds=60
+bkcZKNumRetries=20
+bkcZKRetryStartBackoffMillis=100
+bkcZKRetryMaxBackoffMillis=200
+
+########################
+# BookKeeper Client Settings
+########################
+
+# bookkeeper client timeouts
+bkcWriteTimeoutSeconds=2
+bkcReadTimeoutSeconds=2
+bkcNumWorkerThreads=32
+bkc.numChannelsPerBookie=1
+bkc.enableTaskExecutionStats=true
+bkc.connectTimeoutMillis=200
+bkc.enableParallelRecoveryRead=true
+bkc.recoveryReadBatchSize=5
+bkc.enablePerHostStats=true
+
+########################
+# DL Settings
+########################
+
+# Metadata Settings
+
+# ledger metadata version that supports sequence id
+ledger-metadata-layout=5
+
+# lock timeout
+lockTimeoutSeconds=0
+# dl worker threads
+numWorkerThreads=32
+
+### Recovery Related Settings
+
+# recover log segments in background
+recoverLogSegmentsInBackground=false
+# disable max id in proxy
+maxIdSanityCheck=false
+# use allocator pool for proxy
+enableLedgerAllocatorPool=true
+# ledger allocator pool path
+ledgerAllocatorPoolPath=.write_proxy_allocation_pool
+# ledger allocator pool size
+ledgerAllocatorPoolCoreSize=40
+# check stream exists or not
+createStreamIfNotExists=true
+# encode dc id in version
+encodeDCIDInVersion=true
+# logSegmentNameVersion
+logSegmentNameVersion=1
+
+### Write Performance Related Settings
+
+# ensemble size
+ensemble-size=3
+write-quorum-size=3
+ack-quorum-size=2
+bkc.ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
+bkc.delayEnsembleChange=true
+bkc.writeRequestToChannelAsync=true
+
+# enable immediate flush
+enableImmediateFlush=true
+# 0k output buffer
+output-buffer-size=0
+# disable periodical flush
+periodicFlushFrequencyMilliSeconds=0
+enableTaskExecutionStats=true
+taskExecutionWarnTimeMicros=100000
+
+### Ledger Rolling Related Settings
+
+# retention policy
+retention-size=4
+# rolling ledgers (enable time rolling): 120 minutes = 2 hours
+rolling-interval=120
+# max logsegment bytes : 2GB
+maxLogSegmentBytes=2147483648
+# rolling concurrency
+logSegmentRollingConcurrency=1
+# disable sanityCheckDelete
+sanityCheckDelete=false
+# compression codec
+compressionType=lz4
+
+### Per Stream Stats
+enablePerStreamStat=true
+
+########################
+# DL Settings
+########################
+
+# proxy server settings
+server_mode=DURABLE
+serviceTimeoutMs=60000
+streamProbationTimeoutMs=120000
+server_threads=16
+server_dlsn_version=1
+server_enable_perstream_stat=true
+server_graceful_shutdown_period_ms=20000
+
+# write limits
+perWriterOutstandingWriteLimit=-1
+globalOutstandingWriteLimit=-1
+outstandingWriteLimitDarkmode=false
+
+# bytes per second limit applied at the host level (50MBps on 1Gib machines)
+bpsHardServiceLimit=52428800
+# bytes per second limit after which no new streams may be acquired (65MBps on 1Gib machines)
+bpsStreamAcquireServiceLimit=47185920
+
+# limit the maximum number of streams
+maxAcquiredPartitionsPerProxy=-1

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/conf/zookeeper.conf.template
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/conf/zookeeper.conf.template b/distributedlog-proxy-server/conf/zookeeper.conf.template
new file mode 100644
index 0000000..3c0546e
--- /dev/null
+++ b/distributedlog-proxy-server/conf/zookeeper.conf.template
@@ -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.
+
+# The number of milliseconds of each tick
+tickTime=2000
+
+# the port at which the clients will connect
+clientPort=2181
+
+# The number of ticks that the initial
+# synchronization phase can take
+initLimit=10
+
+# The number of ticks that can pass between
+# sending a request and getting an acknowledgement
+syncLimit=30
+
+# the directory where the snapshot is stored.
+dataDir=/tmp/data/zookeeper
+
+# where txlog  are written
+dataLogDir=/tmp/data/zookeeper/txlog
+
+# the port at which the admin will listen
+adminPort=9990
+zookeeper.admin.enableServer=true
+
+# limit on queued clients - default: 1000
+globalOutstandingLimit=1000
+
+# number of transactions before snapshots are taken - default: 100000
+snapCount=100000
+
+# max # of clients - 0==unlimited
+maxClientCnxns=25
+
+# Election implementation to use. A value of "0" corresponds to the original
+# UDP-based version, "1" corresponds to the non-authenticated UDP-based
+# version of fast leader election, "2" corresponds to the authenticated
+# UDP-based version of fast leader election, and "3" corresponds to TCP-based
+# version of fast leader election. Currently, only 0 and 3 are supported,
+# 3 being the default
+electionAlg=3
+
+# Leader accepts client connections. Default value is "yes". The leader
+# machine coordinates updates. For higher update throughput at thes slight
+# expense of read throughput the leader can be configured to not accept
+# clients and focus on coordination.
+leaderServes=yes
+
+# Skips ACL checks. This results in a boost in throughput, but opens up full
+# access to the data tree to everyone.
+skipACL=no
+
+# Purge txn logs every hour. Before 3.4.x this was done with an external cron
+# job, now we can do it internally.
+autopurge.purgeInterval=1
+
+# Prior to version 3.4 ZooKeeper has always used NIO directly, however in
+# versions 3.4 and later Netty is supported as an option to NIO (replaces).
+# serverCnxnFactory=org.apache.zookeeper.server.NIOServerCnxnFactory
+
+standaloneEnabled=false
+# ZooKeeper Dynamic Reconfiguration
+# See: https://zookeeper.apache.org/doc/trunk/zookeeperReconfig.html
+#
+# standaloneEnabled=false
+# dynamicConfigFile=/path/to/zoo.cfg.dynamic
+#
+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/c44e0278/distributedlog-proxy-server/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/pom.xml b/distributedlog-proxy-server/pom.xml
new file mode 100644
index 0000000..d7cbd56
--- /dev/null
+++ b/distributedlog-proxy-server/pom.xml
@@ -0,0 +1,275 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.distributedlog</groupId>
+    <artifactId>distributedlog</artifactId>
+    <version>0.5.0-incubating-SNAPSHOT</version>
+  </parent>
+  <artifactId>distributedlog-proxy-server</artifactId>
+  <name>Apache DistributedLog :: Proxy Service</name>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-proxy-client</artifactId>
+      <version>${project.parent.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.zookeeper</groupId>
+          <artifactId>zookeeper</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-proxy-protocol</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-core</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-ostrich4_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-thriftmux_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-core</artifactId>
+      <version>${project.parent.version}</version>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+      <version>${zookeeper.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <version>${mockito.version}</version>
+      <scope>test</scope>
+    </dependency> 
+    <dependency>
+      <groupId>org.apache.bookkeeper.stats</groupId>
+      <artifactId>codahale-metrics-provider</artifactId>
+      <version>${bookkeeper.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-servlets</artifactId>
+      <version>${codahale.metrics.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+      <version>${jetty.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+      <version>${jetty.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libthrift</artifactId>
+      <version>${libthrift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>scrooge-core_2.11</artifactId>
+      <version>${scrooge.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>${slf4j.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <version>${curator.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.zookeeper</groupId>
+          <artifactId>zookeeper</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-protocol</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>com.twitter</groupId>
+        <artifactId>scrooge-maven-plugin</artifactId>
+        <version>${scrooge-maven-plugin.version}</version>
+        <configuration>
+          <language>java</language>
+        </configuration>
+        <executions>
+          <execution>
+            <id>thrift-sources</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven-assembly-plugin.version}</version>
+        <configuration>
+          <descriptors>
+            <descriptor>../src/assemble/bin.xml</descriptor>
+          </descriptors>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
+          <forkMode>always</forkMode>
+          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>org.apache.distributedlog.TimedOutTestsListener</value>
+            </property>
+          </properties>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/src/main/resources/findbugsExclude.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${basedir}/lib</outputDirectory>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven-checkstyle-plugin.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>com.puppycrawl.tools</groupId>
+            <artifactId>checkstyle</artifactId>
+            <version>${puppycrawl.checkstyle.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.distributedlog</groupId>
+            <artifactId>distributedlog-build-tools</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <consoleOutput>true</consoleOutput>
+          <failOnViolation>true</failOnViolation>
+          <includeResources>false</includeResources>
+          <includeTestSourceDirectory>true</includeTestSourceDirectory>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>test-compile</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+  <profiles>
+    <profile>
+      <id>twitter-ostrich-provider</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.bookkeeper.stats</groupId>
+          <artifactId>twitter-ostrich-provider</artifactId>
+          <version>${bookkeeper.version}</version>
+          <exclusions>
+            <exclusion>
+              <groupId>com.twitter</groupId>
+              <artifactId>ostrich_2.10</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.twitter</groupId>
+              <artifactId>ostrich_2.9.2</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.java b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.java
new file mode 100644
index 0000000..8db3e90
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.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.bookkeeper.stats;
+
+import com.codahale.metrics.health.HealthCheckRegistry;
+import org.apache.commons.configuration.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extend the codahale metrics provider to run servlets.
+ */
+public class CodahaleMetricsServletProvider extends CodahaleMetricsProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(CodahaleMetricsServletProvider.class);
+
+    ServletReporter servletReporter = null;
+    private final HealthCheckRegistry healthCheckRegistry = new HealthCheckRegistry();
+
+    @Override
+    public void start(Configuration conf) {
+        super.start(conf);
+        Integer httpPort = conf.getInteger("codahaleStatsHttpPort", null);
+        if (null != httpPort) {
+            servletReporter = new ServletReporter(
+                    getMetrics(),
+                    healthCheckRegistry,
+                    httpPort);
+            try {
+                servletReporter.start();
+            } catch (Exception e) {
+                logger.warn("Encountered error on starting the codahale metrics servlet", e);
+            }
+        }
+    }
+
+    @Override
+    public void stop() {
+        if (null != servletReporter) {
+            try {
+                servletReporter.stop();
+            } catch (Exception e) {
+                logger.error("Encountered error on stopping the codahale metrics servlet", e);
+            }
+        }
+        super.stop();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.java b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.java
new file mode 100644
index 0000000..348787a
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.bookkeeper.stats;
+
+import com.codahale.metrics.health.HealthCheckRegistry;
+import com.codahale.metrics.servlets.HealthCheckServlet;
+
+/**
+ * Health Check Servlet Listener.
+ */
+public class HealthCheckServletContextListener extends HealthCheckServlet.ContextListener {
+
+    private final HealthCheckRegistry healthCheckRegistry;
+
+    public HealthCheckServletContextListener(HealthCheckRegistry healthCheckRegistry) {
+        this.healthCheckRegistry = healthCheckRegistry;
+    }
+
+    @Override
+    protected HealthCheckRegistry getHealthCheckRegistry() {
+        return healthCheckRegistry;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.java b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.java
new file mode 100644
index 0000000..15279fe
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.bookkeeper.stats;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.servlets.MetricsServlet;
+
+/**
+ * A servlet to report metrics.
+ */
+public class MetricsServletContextListener extends MetricsServlet.ContextListener {
+
+    private final MetricRegistry metricRegistry;
+
+    public MetricsServletContextListener(MetricRegistry metricRegistry) {
+        this.metricRegistry = metricRegistry;
+    }
+
+    @Override
+    protected MetricRegistry getMetricRegistry() {
+        return metricRegistry;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/ServletReporter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/ServletReporter.java b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/ServletReporter.java
new file mode 100644
index 0000000..267f75a
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/ServletReporter.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.bookkeeper.stats;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.health.HealthCheckRegistry;
+import com.codahale.metrics.servlets.AdminServlet;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+/**
+ * Starts a jetty server on a configurable port to export stats.
+ */
+public class ServletReporter {
+
+    private final MetricRegistry metricRegistry;
+    private final HealthCheckRegistry healthCheckRegistry;
+    private final int port;
+    private final Server jettyServer;
+
+    public ServletReporter(MetricRegistry metricRegistry,
+                           HealthCheckRegistry healthCheckRegistry,
+                           int port) {
+        this.metricRegistry = metricRegistry;
+        this.healthCheckRegistry = healthCheckRegistry;
+        this.port = port;
+        this.jettyServer = new Server(port);
+    }
+
+    public void start() throws Exception {
+        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+        context.setContextPath("/");
+        jettyServer.setHandler(context);
+
+        context.addEventListener(new HealthCheckServletContextListener(healthCheckRegistry));
+        context.addEventListener(new MetricsServletContextListener(metricRegistry));
+        context.addServlet(new ServletHolder(new AdminServlet()), "/*");
+
+        jettyServer.start();
+    }
+
+    public void stop() throws Exception {
+        jettyServer.stop();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/stats/package-info.java
new file mode 100644
index 0000000..5bdb3ce
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/bookkeeper/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.
+ */
+/**
+ * Extension of {@link org.apache.bookkeeper.stats.CodahaleMetricsProvider}.
+ */
+package org.apache.bookkeeper.stats;



[09/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/ServletReporter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/ServletReporter.java b/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/ServletReporter.java
deleted file mode 100644
index 267f75a..0000000
--- a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/ServletReporter.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 org.apache.bookkeeper.stats;
-
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.health.HealthCheckRegistry;
-import com.codahale.metrics.servlets.AdminServlet;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
-
-/**
- * Starts a jetty server on a configurable port to export stats.
- */
-public class ServletReporter {
-
-    private final MetricRegistry metricRegistry;
-    private final HealthCheckRegistry healthCheckRegistry;
-    private final int port;
-    private final Server jettyServer;
-
-    public ServletReporter(MetricRegistry metricRegistry,
-                           HealthCheckRegistry healthCheckRegistry,
-                           int port) {
-        this.metricRegistry = metricRegistry;
-        this.healthCheckRegistry = healthCheckRegistry;
-        this.port = port;
-        this.jettyServer = new Server(port);
-    }
-
-    public void start() throws Exception {
-        ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
-        context.setContextPath("/");
-        jettyServer.setHandler(context);
-
-        context.addEventListener(new HealthCheckServletContextListener(healthCheckRegistry));
-        context.addEventListener(new MetricsServletContextListener(metricRegistry));
-        context.addServlet(new ServletHolder(new AdminServlet()), "/*");
-
-        jettyServer.start();
-    }
-
-    public void stop() throws Exception {
-        jettyServer.stop();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/package-info.java b/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/package-info.java
deleted file mode 100644
index 5bdb3ce..0000000
--- a/distributedlog-service/src/main/java/org/apache/bookkeeper/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.
- */
-/**
- * Extension of {@link org.apache.bookkeeper.stats.CodahaleMetricsProvider}.
- */
-package org.apache.bookkeeper.stats;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/ClientUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ClientUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ClientUtils.java
deleted file mode 100644
index 96bc338..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ClientUtils.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 org.apache.distributedlog.service;
-
-import org.apache.distributedlog.client.DistributedLogClientImpl;
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * DistributedLog Client Related Utils.
- */
-public class ClientUtils {
-
-    public static Pair<DistributedLogClient, MonitorServiceClient> buildClient(DistributedLogClientBuilder builder) {
-        DistributedLogClientImpl clientImpl = builder.buildClient();
-        return Pair.of((DistributedLogClient) clientImpl, (MonitorServiceClient) clientImpl);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
deleted file mode 100644
index 9cc085d..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
+++ /dev/null
@@ -1,352 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.DistributedLogConfiguration;
-import org.apache.distributedlog.LocalDLMEmulator;
-import org.apache.distributedlog.client.routing.SingleHostRoutingService;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.metadata.DLMetadata;
-import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
-import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import com.twitter.finagle.builder.Server;
-import java.io.File;
-import java.net.BindException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * DistributedLog Cluster is an emulator to run distributedlog components.
- */
-public class DistributedLogCluster {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DistributedLogCluster.class);
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder to build distributedlog cluster.
-     */
-    public static class Builder {
-
-        int numBookies = 3;
-        boolean shouldStartZK = true;
-        String zkHost = "127.0.0.1";
-        int zkPort = 0;
-        boolean shouldStartProxy = true;
-        int proxyPort = 7000;
-        boolean thriftmux = false;
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration()
-                .setLockTimeout(10)
-                .setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true);
-        ServerConfiguration bkConf = new ServerConfiguration();
-
-        private Builder() {}
-
-        /**
-         * How many bookies to run. By default is 3.
-         *
-         * @return builder
-         */
-        public Builder numBookies(int numBookies) {
-            this.numBookies = numBookies;
-            return this;
-        }
-
-        /**
-         * Whether to start zookeeper? By default is true.
-         *
-         * @param startZK
-         *          flag to start zookeeper?
-         * @return builder
-         */
-        public Builder shouldStartZK(boolean startZK) {
-            this.shouldStartZK = startZK;
-            return this;
-        }
-
-        /**
-         * ZooKeeper server to run. By default it runs locally on '127.0.0.1'.
-         *
-         * @param zkServers
-         *          zk servers
-         * @return builder
-         */
-        public Builder zkServers(String zkServers) {
-            this.zkHost = zkServers;
-            return this;
-        }
-
-        /**
-         * ZooKeeper server port to listen on. By default it listens on 2181.
-         *
-         * @param zkPort
-         *          zookeeper server port.
-         * @return builder.
-         */
-        public Builder zkPort(int zkPort) {
-            this.zkPort = zkPort;
-            return this;
-        }
-
-        /**
-         * Whether to start proxy or not. By default is true.
-         *
-         * @param startProxy
-         *          whether to start proxy or not.
-         * @return builder
-         */
-        public Builder shouldStartProxy(boolean startProxy) {
-            this.shouldStartProxy = startProxy;
-            return this;
-        }
-
-        /**
-         * Port that proxy server to listen on. By default is 7000.
-         *
-         * @param proxyPort
-         *          port that proxy server to listen on.
-         * @return builder
-         */
-        public Builder proxyPort(int proxyPort) {
-            this.proxyPort = proxyPort;
-            return this;
-        }
-
-        /**
-         * Set the distributedlog configuration.
-         *
-         * @param dlConf
-         *          distributedlog configuration
-         * @return builder
-         */
-        public Builder dlConf(DistributedLogConfiguration dlConf) {
-            this.dlConf = dlConf;
-            return this;
-        }
-
-        /**
-         * Set the Bookkeeper server configuration.
-         *
-         * @param bkConf
-         *          bookkeeper server configuration
-         * @return builder
-         */
-        public Builder bkConf(ServerConfiguration bkConf) {
-            this.bkConf = bkConf;
-            return this;
-        }
-
-        /**
-         * Enable thriftmux for the dl server.
-         *
-         * @param enabled flag to enable thriftmux
-         * @return builder
-         */
-        public Builder thriftmux(boolean enabled) {
-            this.thriftmux = enabled;
-            return this;
-        }
-
-        public DistributedLogCluster build() throws Exception {
-            // build the cluster
-            return new DistributedLogCluster(
-                dlConf,
-                bkConf,
-                numBookies,
-                shouldStartZK,
-                zkHost,
-                zkPort,
-                shouldStartProxy,
-                proxyPort,
-                thriftmux);
-        }
-    }
-
-    /**
-     * Run a distributedlog proxy server.
-     */
-    public static class DLServer {
-
-        static final int MAX_RETRIES = 20;
-        static final int MIN_PORT = 1025;
-        static final int MAX_PORT = 65535;
-
-        int proxyPort;
-
-        public final InetSocketAddress address;
-        public final Pair<DistributedLogServiceImpl, Server> dlServer;
-        private final SingleHostRoutingService routingService = SingleHostRoutingService.of(null);
-
-        protected DLServer(DistributedLogConfiguration dlConf,
-                           URI uri,
-                           int basePort,
-                           boolean thriftmux) throws Exception {
-            proxyPort = basePort;
-
-            boolean success = false;
-            int retries = 0;
-            Pair<DistributedLogServiceImpl, Server> serverPair = null;
-            while (!success) {
-                try {
-                    org.apache.distributedlog.service.config.ServerConfiguration serverConf =
-                            new org.apache.distributedlog.service.config.ServerConfiguration();
-                    serverConf.loadConf(dlConf);
-                    serverConf.setServerShardId(proxyPort);
-                    serverPair = DistributedLogServer.runServer(
-                            serverConf,
-                            dlConf,
-                            uri,
-                            new IdentityStreamPartitionConverter(),
-                            routingService,
-                            new NullStatsProvider(),
-                            proxyPort,
-                            thriftmux,
-                            new EqualLoadAppraiser());
-                    routingService.setAddress(DLSocketAddress.getSocketAddress(proxyPort));
-                    routingService.startService();
-                    serverPair.getLeft().startPlacementPolicy();
-                    success = true;
-                } catch (BindException be) {
-                    retries++;
-                    if (retries > MAX_RETRIES) {
-                        throw be;
-                    }
-                    proxyPort++;
-                    if (proxyPort > MAX_PORT) {
-                        proxyPort = MIN_PORT;
-                    }
-                }
-            }
-
-            LOG.info("Running DL on port {}", proxyPort);
-
-            dlServer = serverPair;
-            address = DLSocketAddress.getSocketAddress(proxyPort);
-        }
-
-        public InetSocketAddress getAddress() {
-            return address;
-        }
-
-        public void shutdown() {
-            DistributedLogServer.closeServer(dlServer, 0, TimeUnit.MILLISECONDS);
-            routingService.stopService();
-        }
-    }
-
-    private final DistributedLogConfiguration dlConf;
-    private final ZooKeeperServerShim zks;
-    private final LocalDLMEmulator dlmEmulator;
-    private DLServer dlServer;
-    private final boolean shouldStartProxy;
-    private final int proxyPort;
-    private final boolean thriftmux;
-    private final List<File> tmpDirs = new ArrayList<File>();
-
-    private DistributedLogCluster(DistributedLogConfiguration dlConf,
-                                  ServerConfiguration bkConf,
-                                  int numBookies,
-                                  boolean shouldStartZK,
-                                  String zkServers,
-                                  int zkPort,
-                                  boolean shouldStartProxy,
-                                  int proxyPort,
-                                  boolean thriftmux) throws Exception {
-        this.dlConf = dlConf;
-        if (shouldStartZK) {
-            File zkTmpDir = IOUtils.createTempDir("zookeeper", "distrlog");
-            tmpDirs.add(zkTmpDir);
-            if (0 == zkPort) {
-                Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkTmpDir);
-                this.zks = serverAndPort.getLeft();
-                zkPort = serverAndPort.getRight();
-            } else {
-                this.zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir);
-            }
-        } else {
-            this.zks = null;
-        }
-        this.dlmEmulator = LocalDLMEmulator.newBuilder()
-                .numBookies(numBookies)
-                .zkHost(zkServers)
-                .zkPort(zkPort)
-                .serverConf(bkConf)
-                .shouldStartZK(false)
-                .build();
-        this.shouldStartProxy = shouldStartProxy;
-        this.proxyPort = proxyPort;
-        this.thriftmux = thriftmux;
-    }
-
-    public void start() throws Exception {
-        this.dlmEmulator.start();
-        BKDLConfig bkdlConfig = new BKDLConfig(this.dlmEmulator.getZkServers(), "/ledgers").setACLRootPath(".acl");
-        DLMetadata.create(bkdlConfig).update(this.dlmEmulator.getUri());
-        if (shouldStartProxy) {
-            this.dlServer = new DLServer(
-                    dlConf,
-                    this.dlmEmulator.getUri(),
-                    proxyPort,
-                    thriftmux);
-        } else {
-            this.dlServer = null;
-        }
-    }
-
-    public void stop() throws Exception {
-        if (null != dlServer) {
-            this.dlServer.shutdown();
-        }
-        this.dlmEmulator.teardown();
-        if (null != this.zks) {
-            this.zks.stop();
-        }
-        for (File dir : tmpDirs) {
-            FileUtils.forceDeleteOnExit(dir);
-        }
-    }
-
-    public URI getUri() {
-        return this.dlmEmulator.getUri();
-    }
-
-    public String getZkServers() {
-        return this.dlmEmulator.getZkServers();
-    }
-
-    public String getProxyFinagleStr() {
-        return "inet!" + (dlServer == null ? "127.0.0.1:" + proxyPort : dlServer.getAddress().toString());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
deleted file mode 100644
index 81e476b..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
+++ /dev/null
@@ -1,460 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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 com.google.common.base.Optional;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.config.DynamicConfigurationFactory;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.service.announcer.Announcer;
-import org.apache.distributedlog.service.announcer.NOPAnnouncer;
-import org.apache.distributedlog.service.announcer.ServerSetAnnouncer;
-import org.apache.distributedlog.service.config.DefaultStreamConfigProvider;
-import org.apache.distributedlog.service.config.NullStreamConfigProvider;
-import org.apache.distributedlog.service.config.ServerConfiguration;
-import org.apache.distributedlog.service.config.ServiceStreamConfigProvider;
-import org.apache.distributedlog.service.config.StreamConfigProvider;
-import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
-import org.apache.distributedlog.service.placement.LoadAppraiser;
-import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.distributedlog.thrift.service.DistributedLogService;
-import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.SchedulerUtils;
-import com.twitter.finagle.Stack;
-import com.twitter.finagle.ThriftMuxServer$;
-import com.twitter.finagle.builder.Server;
-import com.twitter.finagle.builder.ServerBuilder;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientIdRequiredFilter;
-import com.twitter.finagle.thrift.ThriftServerFramedCodec;
-import com.twitter.finagle.transport.Transport;
-import com.twitter.util.Duration;
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-import scala.Tuple2;
-
-/**
- * Running the distributedlog proxy server.
- */
-public class DistributedLogServer {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServer.class);
-    private static final String DEFAULT_LOAD_APPRIASER = EqualLoadAppraiser.class.getCanonicalName();
-
-    private DistributedLogServiceImpl dlService = null;
-    private Server server = null;
-    private RoutingService routingService;
-    private StatsProvider statsProvider;
-    private Announcer announcer = null;
-    private ScheduledExecutorService configExecutorService;
-    private long gracefulShutdownMs = 0L;
-
-    private final StatsReceiver statsReceiver;
-    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-    private final Optional<String> uri;
-    private final Optional<String> conf;
-    private final Optional<String> streamConf;
-    private final Optional<Integer> port;
-    private final Optional<Integer> statsPort;
-    private final Optional<Integer> shardId;
-    private final Optional<Boolean> announceServerSet;
-    private final Optional<String> loadAppraiserClassStr;
-    private final Optional<Boolean> thriftmux;
-
-    DistributedLogServer(Optional<String> uri,
-                         Optional<String> conf,
-                         Optional<String> streamConf,
-                         Optional<Integer> port,
-                         Optional<Integer> statsPort,
-                         Optional<Integer> shardId,
-                         Optional<Boolean> announceServerSet,
-                         Optional<String> loadAppraiserClass,
-                         Optional<Boolean> thriftmux,
-                         RoutingService routingService,
-                         StatsReceiver statsReceiver,
-                         StatsProvider statsProvider) {
-        this.uri = uri;
-        this.conf = conf;
-        this.streamConf = streamConf;
-        this.port = port;
-        this.statsPort = statsPort;
-        this.shardId = shardId;
-        this.announceServerSet = announceServerSet;
-        this.thriftmux = thriftmux;
-        this.routingService = routingService;
-        this.statsReceiver = statsReceiver;
-        this.statsProvider = statsProvider;
-        this.loadAppraiserClassStr = loadAppraiserClass;
-    }
-
-    public void runServer()
-        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
-        if (!uri.isPresent()) {
-            throw new IllegalArgumentException("No distributedlog uri provided.");
-        }
-        URI dlUri = URI.create(uri.get());
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-        if (conf.isPresent()) {
-            String configFile = conf.get();
-            try {
-                dlConf.loadConf(new File(configFile).toURI().toURL());
-            } catch (ConfigurationException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
-                    + configFile + ".");
-            } catch (MalformedURLException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
-                        + configFile + ".");
-            }
-        }
-
-        this.configExecutorService = Executors.newScheduledThreadPool(1,
-                new ThreadFactoryBuilder()
-                        .setNameFormat("DistributedLogService-Dyncfg-%d")
-                        .setDaemon(true)
-                        .build());
-
-        // server configuration and dynamic configuration
-        ServerConfiguration serverConf = new ServerConfiguration();
-        serverConf.loadConf(dlConf);
-
-        // overwrite the shard id if it is provided in the args
-        if (shardId.isPresent()) {
-            serverConf.setServerShardId(shardId.get());
-        }
-
-        serverConf.validate();
-
-        DynamicDistributedLogConfiguration dynDlConf = getServiceDynConf(dlConf);
-
-        logger.info("Starting stats provider : {}", statsProvider.getClass());
-        statsProvider.start(dlConf);
-
-        if (announceServerSet.isPresent() && announceServerSet.get()) {
-            announcer = new ServerSetAnnouncer(
-                    dlUri,
-                    port.or(0),
-                    statsPort.or(0),
-                    shardId.or(0));
-        } else {
-            announcer = new NOPAnnouncer();
-        }
-
-        // Build the stream partition converter
-        StreamPartitionConverter converter;
-        try {
-            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
-        } catch (ConfigurationException e) {
-            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
-                    IdentityStreamPartitionConverter.class.getName());
-            converter = new IdentityStreamPartitionConverter();
-        }
-        Class loadAppraiserClass = Class.forName(loadAppraiserClassStr.or(DEFAULT_LOAD_APPRIASER));
-        LoadAppraiser loadAppraiser = (LoadAppraiser) ReflectionUtils.newInstance(loadAppraiserClass);
-        logger.info("Load appraiser class is " + loadAppraiserClassStr.or("not specified.") + " Instantiated "
-                + loadAppraiser.getClass().getCanonicalName());
-
-        StreamConfigProvider streamConfProvider =
-                getStreamConfigProvider(dlConf, converter);
-
-        // pre-run
-        preRun(dlConf, serverConf);
-
-        Pair<DistributedLogServiceImpl, Server> serverPair = runServer(
-                serverConf,
-                dlConf,
-                dynDlConf,
-                dlUri,
-                converter,
-                routingService,
-                statsProvider,
-                port.or(0),
-                keepAliveLatch,
-                statsReceiver,
-                thriftmux.isPresent(),
-                streamConfProvider,
-                loadAppraiser);
-
-        this.dlService = serverPair.getLeft();
-        this.server = serverPair.getRight();
-
-        // announce the service
-        announcer.announce();
-        // start the routing service after announced
-        routingService.startService();
-        logger.info("Started the routing service.");
-        dlService.startPlacementPolicy();
-        logger.info("Started the placement policy.");
-    }
-
-    protected void preRun(DistributedLogConfiguration conf, ServerConfiguration serverConf) {
-        this.gracefulShutdownMs = serverConf.getGracefulShutdownPeriodMs();
-        if (!serverConf.isDurableWriteEnabled()) {
-            conf.setDurableWriteEnabled(false);
-        }
-    }
-
-    private DynamicDistributedLogConfiguration getServiceDynConf(DistributedLogConfiguration dlConf)
-        throws ConfigurationException {
-        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.absent();
-        if (conf.isPresent()) {
-            DynamicConfigurationFactory configFactory = new DynamicConfigurationFactory(
-                    configExecutorService, dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
-            dynConf = configFactory.getDynamicConfiguration(conf.get());
-        }
-        if (dynConf.isPresent()) {
-            return dynConf.get();
-        } else {
-            return ConfUtils.getConstDynConf(dlConf);
-        }
-    }
-
-    private StreamConfigProvider getStreamConfigProvider(DistributedLogConfiguration dlConf,
-                                                         StreamPartitionConverter partitionConverter)
-            throws ConfigurationException {
-        StreamConfigProvider streamConfProvider = new NullStreamConfigProvider();
-        if (streamConf.isPresent() && conf.isPresent()) {
-            String dynConfigPath = streamConf.get();
-            String defaultConfigFile = conf.get();
-            streamConfProvider = new ServiceStreamConfigProvider(
-                    dynConfigPath,
-                    defaultConfigFile,
-                    partitionConverter,
-                    configExecutorService,
-                    dlConf.getDynamicConfigReloadIntervalSec(),
-                    TimeUnit.SECONDS);
-        } else if (conf.isPresent()) {
-            String configFile = conf.get();
-            streamConfProvider = new DefaultStreamConfigProvider(configFile, configExecutorService,
-                    dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
-        }
-        return streamConfProvider;
-    }
-
-    static Pair<DistributedLogServiceImpl, Server> runServer(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf,
-            URI dlUri,
-            StreamPartitionConverter converter,
-            RoutingService routingService,
-            StatsProvider provider,
-            int port,
-            boolean thriftmux,
-            LoadAppraiser loadAppraiser) throws IOException {
-
-        return runServer(serverConf,
-                dlConf,
-                ConfUtils.getConstDynConf(dlConf),
-                dlUri,
-                converter,
-                routingService,
-                provider,
-                port,
-                new CountDownLatch(0),
-                new NullStatsReceiver(),
-                thriftmux,
-                new NullStreamConfigProvider(),
-                loadAppraiser);
-    }
-
-    static Pair<DistributedLogServiceImpl, Server> runServer(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf,
-            DynamicDistributedLogConfiguration dynDlConf,
-            URI dlUri,
-            StreamPartitionConverter partitionConverter,
-            RoutingService routingService,
-            StatsProvider provider,
-            int port,
-            CountDownLatch keepAliveLatch,
-            StatsReceiver statsReceiver,
-            boolean thriftmux,
-            StreamConfigProvider streamConfProvider,
-            LoadAppraiser loadAppraiser) throws IOException {
-        logger.info("Running server @ uri {}.", dlUri);
-
-        boolean perStreamStatsEnabled = serverConf.isPerStreamStatEnabled();
-        StatsLogger perStreamStatsLogger;
-        if (perStreamStatsEnabled) {
-            perStreamStatsLogger = provider.getStatsLogger("stream");
-        } else {
-            perStreamStatsLogger = NullStatsLogger.INSTANCE;
-        }
-
-        // dl service
-        DistributedLogServiceImpl dlService = new DistributedLogServiceImpl(
-            serverConf,
-            dlConf,
-            dynDlConf,
-            streamConfProvider,
-            dlUri,
-            partitionConverter,
-            routingService,
-            provider.getStatsLogger(""),
-            perStreamStatsLogger,
-            keepAliveLatch,
-            loadAppraiser);
-
-        StatsReceiver serviceStatsReceiver = statsReceiver.scope("service");
-        StatsLogger serviceStatsLogger = provider.getStatsLogger("service");
-
-        ServerBuilder serverBuilder = ServerBuilder.get()
-                .name("DistributedLogServer")
-                .codec(ThriftServerFramedCodec.get())
-                .reportTo(statsReceiver)
-                .keepAlive(true)
-                .bindTo(new InetSocketAddress(port));
-
-        if (thriftmux) {
-            logger.info("Using thriftmux.");
-            Tuple2<Transport.Liveness, Stack.Param<Transport.Liveness>> livenessParam = new Transport.Liveness(
-                    Duration.Top(), Duration.Top(), Option.apply((Object) Boolean.valueOf(true))).mk();
-            serverBuilder = serverBuilder.stack(
-                ThriftMuxServer$.MODULE$.configured(livenessParam._1(), livenessParam._2()));
-        }
-
-        logger.info("DistributedLogServer running with the following configuration : \n{}", dlConf.getPropsAsString());
-
-        // starts dl server
-        Server server = ServerBuilder.safeBuild(
-                new ClientIdRequiredFilter<byte[], byte[]>(serviceStatsReceiver).andThen(
-                    new StatsFilter<byte[], byte[]>(serviceStatsLogger).andThen(
-                        new DistributedLogService.Service(dlService, new TBinaryProtocol.Factory()))),
-                serverBuilder);
-
-        logger.info("Started DistributedLog Server.");
-        return Pair.of(dlService, server);
-    }
-
-    static void closeServer(Pair<DistributedLogServiceImpl, Server> pair,
-                            long gracefulShutdownPeriod,
-                            TimeUnit timeUnit) {
-        if (null != pair.getLeft()) {
-            pair.getLeft().shutdown();
-            if (gracefulShutdownPeriod > 0) {
-                try {
-                    timeUnit.sleep(gracefulShutdownPeriod);
-                } catch (InterruptedException e) {
-                    logger.info("Interrupted on waiting service shutting down state propagated to all clients : ", e);
-                }
-            }
-        }
-        if (null != pair.getRight()) {
-            logger.info("Closing dl thrift server.");
-            pair.getRight().close();
-            logger.info("Closed dl thrift server.");
-        }
-    }
-
-    /**
-     * Close the server.
-     */
-    public void close() {
-        if (null != announcer) {
-            try {
-                announcer.unannounce();
-            } catch (IOException e) {
-                logger.warn("Error on unannouncing service : ", e);
-            }
-            announcer.close();
-        }
-        closeServer(Pair.of(dlService, server), gracefulShutdownMs, TimeUnit.MILLISECONDS);
-        routingService.stopService();
-        if (null != statsProvider) {
-            statsProvider.stop();
-        }
-        SchedulerUtils.shutdownScheduler(configExecutorService, 60, TimeUnit.SECONDS);
-        keepAliveLatch.countDown();
-    }
-
-    public void join() throws InterruptedException {
-        keepAliveLatch.await();
-    }
-
-    /**
-     * Running distributedlog server.
-     *
-     * @param uri distributedlog namespace
-     * @param conf distributedlog configuration file location
-     * @param streamConf per stream configuration dir location
-     * @param port listen port
-     * @param statsPort stats port
-     * @param shardId shard id
-     * @param announceServerSet whether to announce itself to server set
-     * @param thriftmux flag to enable thrift mux
-     * @param statsReceiver receiver to receive finagle stats
-     * @param statsProvider provider to receive dl stats
-     * @return distributedlog server
-     * @throws ConfigurationException
-     * @throws IllegalArgumentException
-     * @throws IOException
-     * @throws ClassNotFoundException
-     */
-    public static DistributedLogServer runServer(
-               Optional<String> uri,
-               Optional<String> conf,
-               Optional<String> streamConf,
-               Optional<Integer> port,
-               Optional<Integer> statsPort,
-               Optional<Integer> shardId,
-               Optional<Boolean> announceServerSet,
-               Optional<String> loadAppraiserClass,
-               Optional<Boolean> thriftmux,
-               RoutingService routingService,
-               StatsReceiver statsReceiver,
-               StatsProvider statsProvider)
-        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
-
-        final DistributedLogServer server = new DistributedLogServer(
-                uri,
-                conf,
-                streamConf,
-                port,
-                statsPort,
-                shardId,
-                announceServerSet,
-                loadAppraiserClass,
-                thriftmux,
-                routingService,
-                statsReceiver,
-                statsProvider);
-
-        server.runServer();
-        return server;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
deleted file mode 100644
index a1642f9..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.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 org.apache.distributedlog.service;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static org.apache.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
-import static org.apache.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
-import static org.apache.distributedlog.util.CommandLineUtils.getOptionalStringArg;
-
-import com.google.common.base.Function;
-import com.google.common.base.Optional;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.client.routing.RoutingUtils;
-import org.apache.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.io.File;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-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.cli.ParseException;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The launcher of the distributedlog proxy server.
- */
-public class DistributedLogServerApp {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServerApp.class);
-
-    private static final String USAGE = "DistributedLogServerApp [-u <uri>] [-c <conf>]";
-    private final String[] args;
-    private final Options options = new Options();
-
-    private DistributedLogServerApp(String[] args) {
-        this.args = args;
-
-        // prepare options
-        options.addOption("u", "uri", true, "DistributedLog URI");
-        options.addOption("c", "conf", true, "DistributedLog Configuration File");
-        options.addOption("sc", "stream-conf", true, "Per Stream Configuration Directory");
-        options.addOption("p", "port", true, "DistributedLog Server Port");
-        options.addOption("sp", "stats-port", true, "DistributedLog Stats Port");
-        options.addOption("pd", "stats-provider", true, "DistributedLog Stats Provider");
-        options.addOption("si", "shard-id", true, "DistributedLog Shard ID");
-        options.addOption("a", "announce", false, "ServerSet Path to Announce");
-        options.addOption("la", "load-appraiser", true, "LoadAppraiser Implementation to Use");
-        options.addOption("mx", "thriftmux", false, "Is thriftmux enabled");
-    }
-
-    private void printUsage() {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(USAGE, options);
-    }
-
-    private void run() {
-        try {
-            logger.info("Running distributedlog server : args = {}", Arrays.toString(args));
-            BasicParser parser = new BasicParser();
-            CommandLine cmdline = parser.parse(options, args);
-            runCmd(cmdline);
-        } catch (ParseException pe) {
-            logger.error("Argument error : {}", pe.getMessage());
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (IllegalArgumentException iae) {
-            logger.error("Argument error : {}", iae.getMessage());
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (ConfigurationException ce) {
-            logger.error("Configuration error : {}", ce.getMessage());
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (IOException ie) {
-            logger.error("Failed to start distributedlog server : ", ie);
-            Runtime.getRuntime().exit(-1);
-        } catch (ClassNotFoundException cnf) {
-          logger.error("Failed to start distributedlog server : ", cnf);
-          Runtime.getRuntime().exit(-1);
-        }
-    }
-
-    private void runCmd(CommandLine cmdline)
-        throws IllegalArgumentException, IOException, ConfigurationException, ClassNotFoundException {
-        final StatsReceiver statsReceiver = NullStatsReceiver.get();
-        Optional<String> confOptional = getOptionalStringArg(cmdline, "c");
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-        if (confOptional.isPresent()) {
-            String configFile = confOptional.get();
-            try {
-                dlConf.loadConf(new File(configFile).toURI().toURL());
-            } catch (ConfigurationException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
-                    + configFile + ".");
-            } catch (MalformedURLException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
-                        + configFile + ".");
-            }
-        }
-        // load the stats provider
-        final StatsProvider statsProvider = getOptionalStringArg(cmdline, "pd")
-                .transform(new Function<String, StatsProvider>() {
-                    @Nullable
-                    @Override
-                    public StatsProvider apply(@Nullable String name) {
-                        return ReflectionUtils.newInstance(name, StatsProvider.class);
-                    }
-                }).or(new NullStatsProvider());
-
-        final Optional<String> uriOption = getOptionalStringArg(cmdline, "u");
-        checkArgument(uriOption.isPresent(), "No distributedlog uri provided.");
-        URI dlUri = URI.create(uriOption.get());
-
-        DLZkServerSet serverSet = DLZkServerSet.of(dlUri, (int) TimeUnit.SECONDS.toMillis(60));
-        RoutingService routingService = RoutingUtils.buildRoutingService(serverSet.getServerSet())
-                .statsReceiver(statsReceiver.scope("routing"))
-                .build();
-
-        final DistributedLogServer server = DistributedLogServer.runServer(
-                uriOption,
-                confOptional,
-                getOptionalStringArg(cmdline, "sc"),
-                getOptionalIntegerArg(cmdline, "p"),
-                getOptionalIntegerArg(cmdline, "sp"),
-                getOptionalIntegerArg(cmdline, "si"),
-                getOptionalBooleanArg(cmdline, "a"),
-                getOptionalStringArg(cmdline, "la"),
-                getOptionalBooleanArg(cmdline, "mx"),
-                routingService,
-                statsReceiver,
-                statsProvider);
-
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                logger.info("Closing DistributedLog Server.");
-                server.close();
-                logger.info("Closed DistributedLog Server.");
-                statsProvider.stop();
-            }
-        });
-
-        try {
-            server.join();
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted when waiting distributedlog server to be finished : ", e);
-        }
-
-        logger.info("DistributedLog Service Interrupted.");
-        server.close();
-        logger.info("Closed DistributedLog Server.");
-        statsProvider.stop();
-    }
-
-    public static void main(String[] args) {
-        final DistributedLogServerApp launcher = new DistributedLogServerApp(args);
-        launcher.run();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
deleted file mode 100644
index c37cd53..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
+++ /dev/null
@@ -1,794 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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 com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.common.net.InetSocketAddressHelper;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
-import org.apache.distributedlog.client.resolver.RegionResolver;
-import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.RegionUnavailableException;
-import org.apache.distributedlog.exceptions.ServiceUnavailableException;
-import org.apache.distributedlog.exceptions.StreamUnavailableException;
-import org.apache.distributedlog.exceptions.TooManyStreamsException;
-import org.apache.distributedlog.feature.AbstractFeatureProvider;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.rate.MovingAverageRate;
-import org.apache.distributedlog.rate.MovingAverageRateFactory;
-import org.apache.distributedlog.service.config.ServerConfiguration;
-import org.apache.distributedlog.service.config.StreamConfigProvider;
-import org.apache.distributedlog.service.placement.LeastLoadPlacementPolicy;
-import org.apache.distributedlog.service.placement.LoadAppraiser;
-import org.apache.distributedlog.service.placement.PlacementPolicy;
-import org.apache.distributedlog.service.placement.ZKPlacementStateManager;
-import org.apache.distributedlog.service.stream.BulkWriteOp;
-import org.apache.distributedlog.service.stream.DeleteOp;
-import org.apache.distributedlog.service.stream.HeartbeatOp;
-import org.apache.distributedlog.service.stream.ReleaseOp;
-import org.apache.distributedlog.service.stream.Stream;
-import org.apache.distributedlog.service.stream.StreamFactory;
-import org.apache.distributedlog.service.stream.StreamFactoryImpl;
-import org.apache.distributedlog.service.stream.StreamManager;
-import org.apache.distributedlog.service.stream.StreamManagerImpl;
-import org.apache.distributedlog.service.stream.StreamOp;
-import org.apache.distributedlog.service.stream.StreamOpStats;
-import org.apache.distributedlog.service.stream.TruncateOp;
-import org.apache.distributedlog.service.stream.WriteOp;
-import org.apache.distributedlog.service.stream.WriteOpWithPayload;
-import org.apache.distributedlog.service.stream.admin.CreateOp;
-import org.apache.distributedlog.service.stream.admin.StreamAdminOp;
-import org.apache.distributedlog.service.stream.limiter.ServiceRequestLimiter;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.distributedlog.service.utils.ServerUtils;
-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.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.ConfUtils;
-import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Await;
-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.ScheduledThreadPoolTimer;
-import com.twitter.util.Timer;
-import java.io.IOException;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-/**
- * Implementation of distributedlog thrift service.
- */
-public class DistributedLogServiceImpl implements DistributedLogService.ServiceIface,
-                                                  FatalErrorHandler {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServiceImpl.class);
-
-    private static final int MOVING_AVERAGE_WINDOW_SECS = 60;
-
-    private final ServerConfiguration serverConfig;
-    private final DistributedLogConfiguration dlConfig;
-    private final DistributedLogNamespace dlNamespace;
-    private final int serverRegionId;
-    private final PlacementPolicy placementPolicy;
-    private ServerStatus serverStatus = ServerStatus.WRITE_AND_ACCEPT;
-    private final ReentrantReadWriteLock closeLock =
-            new ReentrantReadWriteLock();
-    private final CountDownLatch keepAliveLatch;
-    private final byte dlsnVersion;
-    private final String clientId;
-    private final OrderedScheduler scheduler;
-    private final AccessControlManager accessControlManager;
-    private final StreamConfigProvider streamConfigProvider;
-    private final StreamManager streamManager;
-    private final StreamFactory streamFactory;
-    private final RoutingService routingService;
-    private final RegionResolver regionResolver;
-    private final MovingAverageRateFactory movingAvgFactory;
-    private final MovingAverageRate windowedRps;
-    private final MovingAverageRate windowedBps;
-    private final ServiceRequestLimiter limiter;
-    private final Timer timer;
-    private final HashedWheelTimer requestTimer;
-
-    // Features
-    private final FeatureProvider featureProvider;
-    private final Feature featureRegionStopAcceptNewStream;
-    private final Feature featureChecksumDisabled;
-    private final Feature limiterDisabledFeature;
-
-    // Stats
-    private final StatsLogger statsLogger;
-    private final StatsLogger perStreamStatsLogger;
-    private final StreamPartitionConverter streamPartitionConverter;
-    private final StreamOpStats streamOpStats;
-    private final Counter bulkWritePendingStat;
-    private final Counter writePendingStat;
-    private final Counter redirects;
-    private final Counter receivedRecordCounter;
-    private final StatsLogger statusCodeStatLogger;
-    private final ConcurrentHashMap<StatusCode, Counter> statusCodeCounters =
-            new ConcurrentHashMap<StatusCode, Counter>();
-    private final Counter statusCodeTotal;
-    private final Gauge<Number> proxyStatusGauge;
-    private final Gauge<Number> movingAvgRpsGauge;
-    private final Gauge<Number> movingAvgBpsGauge;
-    private final Gauge<Number> streamAcquiredGauge;
-    private final Gauge<Number> streamCachedGauge;
-    private final int shard;
-
-    DistributedLogServiceImpl(ServerConfiguration serverConf,
-                              DistributedLogConfiguration dlConf,
-                              DynamicDistributedLogConfiguration dynDlConf,
-                              StreamConfigProvider streamConfigProvider,
-                              URI uri,
-                              StreamPartitionConverter converter,
-                              RoutingService routingService,
-                              StatsLogger statsLogger,
-                              StatsLogger perStreamStatsLogger,
-                              CountDownLatch keepAliveLatch,
-                              LoadAppraiser loadAppraiser)
-            throws IOException {
-        // Configuration.
-        this.serverConfig = serverConf;
-        this.dlConfig = dlConf;
-        this.perStreamStatsLogger = perStreamStatsLogger;
-        this.dlsnVersion = serverConf.getDlsnVersion();
-        this.serverRegionId = serverConf.getRegionId();
-        this.streamPartitionConverter = converter;
-        int serverPort = serverConf.getServerPort();
-        this.shard = serverConf.getServerShardId();
-        int numThreads = serverConf.getServerThreads();
-        this.clientId = DLSocketAddress.toLockId(DLSocketAddress.getSocketAddress(serverPort), shard);
-        String allocatorPoolName = ServerUtils.getLedgerAllocatorPoolName(
-            serverRegionId,
-            shard,
-            serverConf.isUseHostnameAsAllocatorPoolName());
-        dlConf.setLedgerAllocatorPoolName(allocatorPoolName);
-        this.featureProvider = AbstractFeatureProvider.getFeatureProvider("", dlConf, statsLogger.scope("features"));
-        if (this.featureProvider instanceof AbstractFeatureProvider) {
-            ((AbstractFeatureProvider) featureProvider).start();
-        }
-
-        // Build the namespace
-        this.dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(dlConf)
-                .uri(uri)
-                .statsLogger(statsLogger)
-                .featureProvider(this.featureProvider)
-                .clientId(clientId)
-                .regionId(serverRegionId)
-                .build();
-        this.accessControlManager = this.dlNamespace.createAccessControlManager();
-        this.keepAliveLatch = keepAliveLatch;
-        this.streamConfigProvider = streamConfigProvider;
-
-        // Stats pertaining to stream op execution
-        this.streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-
-        // Executor Service.
-        this.scheduler = OrderedScheduler.newBuilder()
-                .corePoolSize(numThreads)
-                .name("DistributedLogService-Executor")
-                .traceTaskExecution(true)
-                .statsLogger(statsLogger.scope("scheduler"))
-                .build();
-
-        // Timer, kept separate to ensure reliability of timeouts.
-        this.requestTimer = new HashedWheelTimer(
-            new ThreadFactoryBuilder().setNameFormat("DLServiceTimer-%d").build(),
-            dlConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
-            dlConf.getTimeoutTimerNumTicks());
-
-        // Creating and managing Streams
-        this.streamFactory = new StreamFactoryImpl(clientId,
-                streamOpStats,
-                serverConf,
-                dlConf,
-                featureProvider,
-                streamConfigProvider,
-                converter,
-                dlNamespace,
-                scheduler,
-                this,
-                requestTimer);
-        this.streamManager = new StreamManagerImpl(
-                clientId,
-                dlConf,
-                scheduler,
-                streamFactory,
-                converter,
-                streamConfigProvider,
-                dlNamespace);
-        this.routingService = routingService;
-        this.regionResolver = new DefaultRegionResolver();
-
-        // Service features
-        this.featureRegionStopAcceptNewStream = this.featureProvider.getFeature(
-                ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase());
-        this.featureChecksumDisabled = this.featureProvider.getFeature(
-                ServerFeatureKeys.SERVICE_CHECKSUM_DISABLED.name().toLowerCase());
-        this.limiterDisabledFeature = this.featureProvider.getFeature(
-                ServerFeatureKeys.SERVICE_GLOBAL_LIMITER_DISABLED.name().toLowerCase());
-
-        // Resource limiting
-        this.timer = new ScheduledThreadPoolTimer(1, "timer", true);
-        this.movingAvgFactory = new MovingAverageRateFactory(timer);
-        this.windowedRps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
-        this.windowedBps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
-        this.limiter = new ServiceRequestLimiter(
-                dynDlConf,
-                streamOpStats.baseScope("service_limiter"),
-                windowedRps,
-                windowedBps,
-                streamManager,
-                limiterDisabledFeature);
-
-        this.placementPolicy = new LeastLoadPlacementPolicy(
-            loadAppraiser,
-            routingService,
-            dlNamespace,
-            new ZKPlacementStateManager(uri, dlConf, statsLogger),
-            Duration.fromSeconds(serverConf.getResourcePlacementRefreshInterval()),
-            statsLogger);
-        logger.info("placement started");
-
-        // Stats
-        this.statsLogger = statsLogger;
-
-        // Gauges for server status/health
-        this.proxyStatusGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return ServerStatus.DOWN == serverStatus ? -1 : (featureRegionStopAcceptNewStream.isAvailable()
-                    ? 3 : (ServerStatus.WRITE_AND_ACCEPT == serverStatus ? 1 : 2));
-            }
-        };
-        this.movingAvgRpsGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return windowedRps.get();
-            }
-        };
-        this.movingAvgBpsGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return windowedBps.get();
-            }
-        };
-        // Gauges for streams
-        this.streamAcquiredGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return streamManager.numAcquired();
-            }
-        };
-        this.streamCachedGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return streamManager.numCached();
-            }
-        };
-
-        // Stats on server
-        statsLogger.registerGauge("proxy_status", proxyStatusGauge);
-        // Global moving average rps
-        statsLogger.registerGauge("moving_avg_rps", movingAvgRpsGauge);
-        // Global moving average bps
-        statsLogger.registerGauge("moving_avg_bps", movingAvgBpsGauge);
-        // Stats on requests
-        this.bulkWritePendingStat = streamOpStats.requestPendingCounter("bulkWritePending");
-        this.writePendingStat = streamOpStats.requestPendingCounter("writePending");
-        this.redirects = streamOpStats.requestCounter("redirect");
-        this.statusCodeStatLogger = streamOpStats.requestScope("statuscode");
-        this.statusCodeTotal = streamOpStats.requestCounter("statuscode_count");
-        this.receivedRecordCounter = streamOpStats.recordsCounter("received");
-
-        // Stats for streams
-        StatsLogger streamsStatsLogger = statsLogger.scope("streams");
-        streamsStatsLogger.registerGauge("acquired", this.streamAcquiredGauge);
-        streamsStatsLogger.registerGauge("cached", this.streamCachedGauge);
-
-        // Setup complete
-        logger.info("Running distributedlog server : client id {}, allocator pool {}, perstream stat {},"
-            + " dlsn version {}.",
-            new Object[] { clientId, allocatorPoolName, serverConf.isPerStreamStatEnabled(), dlsnVersion });
-    }
-
-    private void countStatusCode(StatusCode code) {
-        Counter counter = statusCodeCounters.get(code);
-        if (null == counter) {
-            counter = statusCodeStatLogger.getCounter(code.name());
-            Counter oldCounter = statusCodeCounters.putIfAbsent(code, counter);
-            if (null != oldCounter) {
-                counter = oldCounter;
-            }
-        }
-        counter.inc();
-        statusCodeTotal.inc();
-    }
-
-    @Override
-    public Future<ServerInfo> handshake() {
-        return handshakeWithClientInfo(new ClientInfo());
-    }
-
-    @Override
-    public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
-        ServerInfo serverInfo = new ServerInfo();
-        closeLock.readLock().lock();
-        try {
-            serverInfo.setServerStatus(serverStatus);
-        } finally {
-            closeLock.readLock().unlock();
-        }
-
-        if (clientInfo.isSetGetOwnerships() && !clientInfo.isGetOwnerships()) {
-            return Future.value(serverInfo);
-        }
-
-        Optional<String> regex = Optional.absent();
-        if (clientInfo.isSetStreamNameRegex()) {
-            regex = Optional.of(clientInfo.getStreamNameRegex());
-        }
-
-        Map<String, String> ownershipMap = streamManager.getStreamOwnershipMap(regex);
-        serverInfo.setOwnerships(ownershipMap);
-        return Future.value(serverInfo);
-    }
-
-    @VisibleForTesting
-    Stream getLogWriter(String stream) throws IOException {
-        Stream writer = streamManager.getStream(stream);
-        if (null == writer) {
-            closeLock.readLock().lock();
-            try {
-                if (featureRegionStopAcceptNewStream.isAvailable()) {
-                    // accept new stream is disabled in current dc
-                    throw new RegionUnavailableException("Region is unavailable right now.");
-                } else if (!(ServerStatus.WRITE_AND_ACCEPT == serverStatus)) {
-                    // if it is closed, we would not acquire stream again.
-                    return null;
-                }
-                writer = streamManager.getOrCreateStream(stream, true);
-            } finally {
-                closeLock.readLock().unlock();
-            }
-        }
-        return writer;
-    }
-
-    // Service interface methods
-
-    @Override
-    public Future<WriteResponse> write(final String stream, ByteBuffer data) {
-        receivedRecordCounter.inc();
-        return doWrite(stream, data, null /* checksum */, false);
-    }
-
-    @Override
-    public Future<BulkWriteResponse> writeBulkWithContext(final String stream,
-                                                          List<ByteBuffer> data,
-                                                          WriteContext ctx) {
-        bulkWritePendingStat.inc();
-        receivedRecordCounter.add(data.size());
-        BulkWriteOp op = new BulkWriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
-            getChecksum(ctx), featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result().ensure(new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                bulkWritePendingStat.dec();
-                return null;
-            }
-        });
-    }
-
-    @Override
-    public Future<WriteResponse> writeWithContext(final String stream, ByteBuffer data, WriteContext ctx) {
-        return doWrite(stream, data, getChecksum(ctx), ctx.isIsRecordSet());
-    }
-
-    @Override
-    public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
-        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> heartbeatWithOptions(String stream, WriteContext ctx, HeartbeatOptions options) {
-        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        if (options.isSendHeartBeatToReader()) {
-            op.setWriteControlRecord(true);
-        }
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> truncate(String stream, String dlsn, WriteContext ctx) {
-        TruncateOp op = new TruncateOp(
-            stream,
-            DLSN.deserialize(dlsn),
-            statsLogger,
-            perStreamStatsLogger,
-            getChecksum(ctx),
-            featureChecksumDisabled,
-            accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> delete(String stream, WriteContext ctx) {
-        DeleteOp op = new DeleteOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> release(String stream, WriteContext ctx) {
-        ReleaseOp op = new ReleaseOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> create(String stream, WriteContext ctx) {
-        CreateOp op = new CreateOp(stream, statsLogger, streamManager, getChecksum(ctx), featureChecksumDisabled);
-        return executeStreamAdminOp(op);
-    }
-
-    //
-    // Ownership RPC
-    //
-
-    @Override
-    public Future<WriteResponse> getOwner(String streamName, WriteContext ctx) {
-        if (streamManager.isAcquired(streamName)) {
-            // the stream is already acquired
-            return Future.value(new WriteResponse(ResponseUtils.ownerToHeader(clientId)));
-        }
-
-        return placementPolicy.placeStream(streamName).map(new Function<String, WriteResponse>() {
-            @Override
-            public WriteResponse apply(String server) {
-                String host = DLSocketAddress.toLockId(InetSocketAddressHelper.parse(server), -1);
-                return new WriteResponse(ResponseUtils.ownerToHeader(host));
-            }
-        });
-    }
-
-
-    //
-    // Admin RPCs
-    //
-
-    @Override
-    public Future<Void> setAcceptNewStream(boolean enabled) {
-        closeLock.writeLock().lock();
-        try {
-            logger.info("Set AcceptNewStream = {}", enabled);
-            if (ServerStatus.DOWN != serverStatus) {
-                if (enabled) {
-                    serverStatus = ServerStatus.WRITE_AND_ACCEPT;
-                } else {
-                    serverStatus = ServerStatus.WRITE_ONLY;
-                }
-            }
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-        return Future.Void();
-    }
-
-    private Future<WriteResponse> doWrite(final String name,
-                                          ByteBuffer data,
-                                          Long checksum,
-                                          boolean isRecordSet) {
-        writePendingStat.inc();
-        receivedRecordCounter.inc();
-        WriteOp op = newWriteOp(name, data, checksum, isRecordSet);
-        executeStreamOp(op);
-        return op.result().ensure(new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                writePendingStat.dec();
-                return null;
-            }
-        });
-    }
-
-    private Long getChecksum(WriteContext ctx) {
-        return ctx.isSetCrc32() ? ctx.getCrc32() : null;
-    }
-
-    private Future<WriteResponse> executeStreamAdminOp(final StreamAdminOp op) {
-        try {
-            op.preExecute();
-        } catch (DLException dle) {
-            return Future.exception(dle);
-        }
-        return op.execute();
-    }
-
-    private void executeStreamOp(final StreamOp op) {
-
-        // Must attach this as early as possible--returning before this point will cause us to
-        // lose the status code.
-        op.responseHeader().addEventListener(new FutureEventListener<ResponseHeader>() {
-            @Override
-            public void onSuccess(ResponseHeader header) {
-                if (header.getLocation() != null || header.getCode() == StatusCode.FOUND) {
-                    redirects.inc();
-                }
-                countStatusCode(header.getCode());
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-            }
-        });
-
-        try {
-            // Apply the request limiter
-            limiter.apply(op);
-
-            // Execute per-op pre-exec code
-            op.preExecute();
-
-        } catch (TooManyStreamsException e) {
-            // Translate to StreamUnavailableException to ensure that the client will redirect
-            // to a different host. Ideally we would be able to return TooManyStreamsException,
-            // but the way exception handling works right now we can't control the handling in
-            // the client because client changes deploy very slowly.
-            op.fail(new StreamUnavailableException(e.getMessage()));
-            return;
-        } catch (Exception e) {
-            op.fail(e);
-            return;
-        }
-
-        Stream stream;
-        try {
-            stream = getLogWriter(op.streamName());
-        } catch (RegionUnavailableException rue) {
-            // redirect the requests to other region
-            op.fail(new RegionUnavailableException("Region " + serverRegionId + " is unavailable."));
-            return;
-        } catch (IOException e) {
-            op.fail(e);
-            return;
-        }
-        if (null == stream) {
-            // redirect the requests when stream is unavailable.
-            op.fail(new ServiceUnavailableException("Server " + clientId + " is closed."));
-            return;
-        }
-
-        if (op instanceof WriteOpWithPayload) {
-            WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
-            windowedBps.add(writeOp.getPayloadSize());
-            windowedRps.inc();
-        }
-
-        stream.submit(op);
-    }
-
-    void shutdown() {
-        try {
-            closeLock.writeLock().lock();
-            try {
-                if (ServerStatus.DOWN == serverStatus) {
-                    return;
-                }
-                serverStatus = ServerStatus.DOWN;
-            } finally {
-                closeLock.writeLock().unlock();
-            }
-
-            streamManager.close();
-            movingAvgFactory.close();
-            limiter.close();
-
-            Stopwatch closeStreamsStopwatch = Stopwatch.createStarted();
-
-            Future<List<Void>> closeResult = streamManager.closeStreams();
-            logger.info("Waiting for closing all streams ...");
-            try {
-                Await.result(closeResult, Duration.fromTimeUnit(5, TimeUnit.MINUTES));
-                logger.info("Closed all streams in {} millis.",
-                        closeStreamsStopwatch.elapsed(TimeUnit.MILLISECONDS));
-            } catch (InterruptedException e) {
-                logger.warn("Interrupted on waiting for closing all streams : ", e);
-                Thread.currentThread().interrupt();
-            } catch (Exception e) {
-                logger.warn("Sorry, we didn't close all streams gracefully in 5 minutes : ", e);
-            }
-
-            // shutdown the dl namespace
-            logger.info("Closing distributedlog namespace ...");
-            dlNamespace.close();
-            logger.info("Closed distributedlog namespace .");
-
-            // Stop the feature provider
-            if (this.featureProvider instanceof AbstractFeatureProvider) {
-                ((AbstractFeatureProvider) featureProvider).stop();
-            }
-
-            // Stop the timer.
-            timer.stop();
-            placementPolicy.close();
-
-            // clean up gauge
-            unregisterGauge();
-
-            // shutdown the executor after requesting closing streams.
-            SchedulerUtils.shutdownScheduler(scheduler, 60, TimeUnit.SECONDS);
-        } catch (Exception ex) {
-            logger.info("Exception while shutting down distributedlog service.");
-        } finally {
-            // release the keepAliveLatch in case shutdown is called from a shutdown hook.
-            keepAliveLatch.countDown();
-            logger.info("Finished shutting down distributedlog service.");
-        }
-    }
-
-    protected void startPlacementPolicy() {
-        this.placementPolicy.start(shard == 0);
-    }
-
-    @Override
-    public void notifyFatalError() {
-        triggerShutdown();
-    }
-
-    private void triggerShutdown() {
-        // release the keepAliveLatch to let the main thread shutdown the whole service.
-        logger.info("Releasing KeepAlive Latch to trigger shutdown ...");
-        keepAliveLatch.countDown();
-        logger.info("Released KeepAlive Latch. Main thread will shut the service down.");
-    }
-
-    // Test methods.
-
-    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
-        Optional<DynamicDistributedLogConfiguration> dynDlConf =
-                streamConfigProvider.getDynamicStreamConfig(streamName);
-        if (dynDlConf.isPresent()) {
-            return dynDlConf.get();
-        } else {
-            return ConfUtils.getConstDynConf(dlConfig);
-        }
-    }
-
-    /**
-     * clean up the gauge before we close to help GC.
-     */
-    private void unregisterGauge(){
-        this.statsLogger.unregisterGauge("proxy_status", this.proxyStatusGauge);
-        this.statsLogger.unregisterGauge("moving_avg_rps", this.movingAvgRpsGauge);
-        this.statsLogger.unregisterGauge("moving_avg_bps", this.movingAvgBpsGauge);
-        this.statsLogger.unregisterGauge("acquired", this.streamAcquiredGauge);
-        this.statsLogger.unregisterGauge("cached", this.streamCachedGauge);
-    }
-
-    @VisibleForTesting
-    Stream newStream(String name) throws IOException {
-        return streamManager.getOrCreateStream(name, false);
-    }
-
-    @VisibleForTesting
-    WriteOp newWriteOp(String stream, ByteBuffer data, Long checksum) {
-        return newWriteOp(stream, data, checksum, false);
-    }
-
-    @VisibleForTesting
-    RoutingService getRoutingService() {
-        return this.routingService;
-    }
-
-    @VisibleForTesting
-    DLSocketAddress getServiceAddress() throws IOException {
-        return DLSocketAddress.deserialize(clientId);
-    }
-
-    WriteOp newWriteOp(String stream,
-                       ByteBuffer data,
-                       Long checksum,
-                       boolean isRecordSet) {
-        return new WriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
-            serverConfig, dlsnVersion, checksum, isRecordSet, featureChecksumDisabled,
-            accessControlManager);
-    }
-
-    @VisibleForTesting
-    Future<List<Void>> closeStreams() {
-        return streamManager.closeStreams();
-    }
-
-    @VisibleForTesting
-    public DistributedLogNamespace getDistributedLogNamespace() {
-        return dlNamespace;
-    }
-
-    @VisibleForTesting
-    StreamManager getStreamManager() {
-        return streamManager;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
deleted file mode 100644
index 17b5ab3..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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;
-
-/**
- * Implement handling for an unrecoverable error.
- */
-public interface FatalErrorHandler {
-
-    /**
-     * This method is invoked when an unrecoverable error has occurred
-     * and no progress can be made. It should implement a shutdown routine.
-     */
-    void notifyFatalError();
-}


[04/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
deleted file mode 100644
index 3f28c42..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.service.stream.admin;
-
-import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.exceptions.ChecksumFailedException;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.service.stream.StreamManager;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.ProtocolUtils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureTransformer;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-
-/**
- * Stream admin op.
- */
-public abstract class StreamAdminOp implements AdminOp<WriteResponse> {
-
-    protected final String stream;
-    protected final StreamManager streamManager;
-    protected final OpStatsLogger opStatsLogger;
-    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
-    protected final Long checksum;
-    protected final Feature checksumDisabledFeature;
-
-    protected StreamAdminOp(String stream,
-                            StreamManager streamManager,
-                            OpStatsLogger statsLogger,
-                            Long checksum,
-                            Feature checksumDisabledFeature) {
-        this.stream = stream;
-        this.streamManager = streamManager;
-        this.opStatsLogger = statsLogger;
-        // start here in case the operation is failed before executing.
-        stopwatch.reset().start();
-        this.checksum = checksum;
-        this.checksumDisabledFeature = checksumDisabledFeature;
-    }
-
-    protected Long computeChecksum() {
-        return ProtocolUtils.streamOpCRC32(stream);
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
-            Long serverChecksum = computeChecksum();
-            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
-                throw new ChecksumFailedException();
-            }
-        }
-    }
-
-    /**
-     * Execute the operation.
-     *
-     * @return execute operation
-     */
-    protected abstract Future<WriteResponse> executeOp();
-
-    @Override
-    public Future<WriteResponse> execute() {
-        return executeOp().transformedBy(new FutureTransformer<WriteResponse, WriteResponse>() {
-
-            @Override
-            public WriteResponse map(WriteResponse response) {
-                opStatsLogger.registerSuccessfulEvent(
-                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                return response;
-            }
-
-            @Override
-            public WriteResponse handle(Throwable cause) {
-                opStatsLogger.registerFailedEvent(
-                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                return ResponseUtils.write(ResponseUtils.exceptionToHeader(cause));
-            }
-
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java
deleted file mode 100644
index 5b583e1..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/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.
- */
-/**
- * Stream Related Admin Operations.
- */
-package org.apache.distributedlog.service.stream.admin;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
deleted file mode 100644
index 5db2037..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream.limiter;
-
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.limiter.RequestLimiter;
-import java.io.Closeable;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.event.ConfigurationEvent;
-import org.apache.commons.configuration.event.ConfigurationListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Dynamically rebuild a rate limiter when the supplied dynamic config changes.
- *
- * <p>Subclasses implement build() to build the limiter. DynamicRequestLimiter must be closed to deregister
- * the config listener.
- */
-public abstract class DynamicRequestLimiter<Req> implements RequestLimiter<Req>, Closeable {
-    private static final Logger LOG = LoggerFactory.getLogger(DynamicRequestLimiter.class);
-
-    private final ConfigurationListener listener;
-    private final Feature rateLimitDisabledFeature;
-    volatile RequestLimiter<Req> limiter;
-    final DynamicDistributedLogConfiguration dynConf;
-
-    public DynamicRequestLimiter(DynamicDistributedLogConfiguration dynConf,
-                                 StatsLogger statsLogger,
-                                 Feature rateLimitDisabledFeature) {
-        final StatsLogger limiterStatsLogger = statsLogger.scope("dynamic");
-        this.dynConf = dynConf;
-        this.rateLimitDisabledFeature = rateLimitDisabledFeature;
-        this.listener = new ConfigurationListener() {
-            @Override
-            public void configurationChanged(ConfigurationEvent event) {
-                // Note that this method may be called several times if several config options
-                // are changed. The effect is harmless except that we create and discard more
-                // objects than we need to.
-                LOG.debug("Config changed callback invoked with event {} {} {} {}", new Object[] {
-                        event.getPropertyName(), event.getPropertyValue(), event.getType(),
-                        event.isBeforeUpdate()});
-                if (!event.isBeforeUpdate()) {
-                    limiterStatsLogger.getCounter("config_changed").inc();
-                    LOG.debug("Rebuilding limiter");
-                    limiter = build();
-                }
-            }
-        };
-        LOG.debug("Registering config changed callback");
-        dynConf.addConfigurationListener(listener);
-    }
-
-    public void initialize() {
-        this.limiter = build();
-    }
-
-    @Override
-    public void apply(Req request) throws OverCapacityException {
-        if (rateLimitDisabledFeature.isAvailable()) {
-            return;
-        }
-        limiter.apply(request);
-    }
-
-    @Override
-    public void close() {
-        boolean success = dynConf.removeConfigurationListener(listener);
-        LOG.debug("Deregistering config changed callback success={}", success);
-    }
-
-   /**
-    * Build the underlying limiter. Called when DynamicRequestLimiter detects config has changed.
-    * This may be called multiple times so the method should be cheap.
-    */
-    protected abstract RequestLimiter<Req> build();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
deleted file mode 100644
index fc30599..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream.limiter;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import org.apache.distributedlog.limiter.GuavaRateLimiter;
-import org.apache.distributedlog.limiter.RateLimiter;
-import org.apache.distributedlog.limiter.RequestLimiter;
-import org.apache.distributedlog.service.stream.StreamOp;
-import org.apache.distributedlog.service.stream.WriteOpWithPayload;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Request limiter builder.
- */
-public class RequestLimiterBuilder {
-    private OverlimitFunction<StreamOp> overlimitFunction = NOP_OVERLIMIT_FUNCTION;
-    private RateLimiter limiter;
-    private CostFunction<StreamOp> costFunction;
-    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-
-    /**
-     * Function to calculate the `RPS` (Request per second) cost of a given stream operation.
-     */
-    public static final CostFunction<StreamOp> RPS_COST_FUNCTION = new CostFunction<StreamOp>() {
-        @Override
-        public int apply(StreamOp op) {
-            if (op instanceof WriteOpWithPayload) {
-                return 1;
-            } else {
-                return 0;
-            }
-        }
-    };
-
-    /**
-     * Function to calculate the `BPS` (Bytes per second) cost of a given stream operation.
-     */
-    public static final CostFunction<StreamOp> BPS_COST_FUNCTION = new CostFunction<StreamOp>() {
-        @Override
-        public int apply(StreamOp op) {
-            if (op instanceof WriteOpWithPayload) {
-                WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
-                return (int) Math.min(writeOp.getPayloadSize(), Integer.MAX_VALUE);
-            } else {
-                return 0;
-            }
-        }
-    };
-
-    /**
-     * Function to check if a stream operation will cause {@link OverCapacityException}.
-     */
-    public static final OverlimitFunction<StreamOp> NOP_OVERLIMIT_FUNCTION = new OverlimitFunction<StreamOp>() {
-        @Override
-        public void apply(StreamOp op) throws OverCapacityException {
-            return;
-        }
-    };
-
-    public RequestLimiterBuilder limit(int limit) {
-        this.limiter = GuavaRateLimiter.of(limit);
-        return this;
-    }
-
-    public RequestLimiterBuilder overlimit(OverlimitFunction<StreamOp> overlimitFunction) {
-        this.overlimitFunction = overlimitFunction;
-        return this;
-    }
-
-    public RequestLimiterBuilder cost(CostFunction<StreamOp> costFunction) {
-        this.costFunction = costFunction;
-        return this;
-    }
-
-    public RequestLimiterBuilder statsLogger(StatsLogger statsLogger) {
-        this.statsLogger = statsLogger;
-        return this;
-    }
-
-    public static RequestLimiterBuilder newRpsLimiterBuilder() {
-        return new RequestLimiterBuilder().cost(RPS_COST_FUNCTION);
-    }
-
-    public static RequestLimiterBuilder newBpsLimiterBuilder() {
-        return new RequestLimiterBuilder().cost(BPS_COST_FUNCTION);
-    }
-
-    public RequestLimiter<StreamOp> build() {
-        checkNotNull(limiter);
-        checkNotNull(overlimitFunction);
-        checkNotNull(costFunction);
-        return new ComposableRequestLimiter(limiter, overlimitFunction, costFunction, statsLogger);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
deleted file mode 100644
index de805aa..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.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 org.apache.distributedlog.service.stream.limiter;
-
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.limiter.ChainedRequestLimiter;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import org.apache.distributedlog.limiter.RequestLimiter;
-import org.apache.distributedlog.rate.MovingAverageRate;
-import org.apache.distributedlog.service.stream.StreamManager;
-import org.apache.distributedlog.service.stream.StreamOp;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Request limiter for the service instance (global request limiter).
- */
-public class ServiceRequestLimiter extends DynamicRequestLimiter<StreamOp> {
-    private final StatsLogger limiterStatLogger;
-    private final MovingAverageRate serviceRps;
-    private final MovingAverageRate serviceBps;
-    private final StreamManager streamManager;
-
-    public ServiceRequestLimiter(DynamicDistributedLogConfiguration dynConf,
-                                 StatsLogger statsLogger,
-                                 MovingAverageRate serviceRps,
-                                 MovingAverageRate serviceBps,
-                                 StreamManager streamManager,
-                                 Feature disabledFeature) {
-        super(dynConf, statsLogger, disabledFeature);
-        this.limiterStatLogger = statsLogger;
-        this.streamManager = streamManager;
-        this.serviceRps = serviceRps;
-        this.serviceBps = serviceBps;
-        this.limiter = build();
-    }
-
-    @Override
-    public RequestLimiter<StreamOp> build() {
-        int rpsStreamAcquireLimit = dynConf.getRpsStreamAcquireServiceLimit();
-        int rpsSoftServiceLimit = dynConf.getRpsSoftServiceLimit();
-        int rpsHardServiceLimit = dynConf.getRpsHardServiceLimit();
-        int bpsStreamAcquireLimit = dynConf.getBpsStreamAcquireServiceLimit();
-        int bpsSoftServiceLimit = dynConf.getBpsSoftServiceLimit();
-        int bpsHardServiceLimit = dynConf.getBpsHardServiceLimit();
-
-        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
-            .limit(rpsHardServiceLimit)
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp request) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for the service instance");
-                }
-            });
-
-        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
-            .limit(rpsSoftServiceLimit);
-
-        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
-            .limit(bpsHardServiceLimit)
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp request) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for the service instance");
-                }
-            });
-
-        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
-            .limit(bpsSoftServiceLimit);
-
-        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
-        builder.addLimiter(new StreamAcquireLimiter(
-            streamManager, serviceRps, rpsStreamAcquireLimit, limiterStatLogger.scope("rps_acquire")));
-        builder.addLimiter(new StreamAcquireLimiter(
-            streamManager, serviceBps, bpsStreamAcquireLimit, limiterStatLogger.scope("bps_acquire")));
-        builder.addLimiter(bpsHardLimiterBuilder.build());
-        builder.addLimiter(bpsSoftLimiterBuilder.build());
-        builder.addLimiter(rpsHardLimiterBuilder.build());
-        builder.addLimiter(rpsSoftLimiterBuilder.build());
-        builder.statsLogger(limiterStatLogger);
-        return builder.build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
deleted file mode 100644
index 7675d6f..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.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 org.apache.distributedlog.service.stream.limiter;
-
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.exceptions.TooManyStreamsException;
-import org.apache.distributedlog.limiter.RequestLimiter;
-import org.apache.distributedlog.rate.MovingAverageRate;
-import org.apache.distributedlog.service.stream.StreamManager;
-import org.apache.distributedlog.service.stream.StreamOp;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * A special limiter on limiting acquiring new streams.
- */
-public class StreamAcquireLimiter implements RequestLimiter<StreamOp> {
-    private final StreamManager streamManager;
-    private final MovingAverageRate serviceRps;
-    private final double serviceRpsLimit;
-    private final Counter overlimitCounter;
-
-    public StreamAcquireLimiter(StreamManager streamManager,
-                                MovingAverageRate serviceRps,
-                                double serviceRpsLimit,
-                                StatsLogger statsLogger) {
-        this.streamManager = streamManager;
-        this.serviceRps = serviceRps;
-        this.serviceRpsLimit = serviceRpsLimit;
-        this.overlimitCounter = statsLogger.getCounter("overlimit");
-    }
-
-    @Override
-    public void apply(StreamOp op) throws OverCapacityException {
-        String streamName = op.streamName();
-        if (serviceRpsLimit > -1 && serviceRps.get() > serviceRpsLimit && !streamManager.isAcquired(streamName)) {
-            overlimitCounter.inc();
-            throw new TooManyStreamsException("Request rate is too high to accept new stream " + streamName + ".");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java
deleted file mode 100644
index 42b4e1e..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.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 org.apache.distributedlog.service.stream.limiter;
-
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.limiter.ChainedRequestLimiter;
-import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import org.apache.distributedlog.limiter.RequestLimiter;
-import org.apache.distributedlog.service.stream.StreamOp;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * A dynamic request limiter on limiting stream operations.
- */
-public class StreamRequestLimiter extends DynamicRequestLimiter<StreamOp> {
-    private final DynamicDistributedLogConfiguration dynConf;
-    private final StatsLogger limiterStatLogger;
-    private final String streamName;
-
-    public StreamRequestLimiter(String streamName,
-                                DynamicDistributedLogConfiguration dynConf,
-                                StatsLogger statsLogger,
-                                Feature disabledFeature) {
-        super(dynConf, statsLogger, disabledFeature);
-        this.limiterStatLogger = statsLogger;
-        this.dynConf = dynConf;
-        this.streamName = streamName;
-        this.limiter = build();
-    }
-
-    @Override
-    public RequestLimiter<StreamOp> build() {
-
-        // RPS hard, soft limits
-        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
-            .limit(dynConf.getRpsHardWriteLimit())
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp op) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for stream " + streamName);
-                }
-            });
-        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
-            .limit(dynConf.getRpsSoftWriteLimit());
-
-        // BPS hard, soft limits
-        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
-            .limit(dynConf.getBpsHardWriteLimit())
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp op) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for stream " + streamName);
-                }
-            });
-        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
-            .limit(dynConf.getBpsSoftWriteLimit());
-
-        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
-        builder.addLimiter(rpsSoftLimiterBuilder.build());
-        builder.addLimiter(rpsHardLimiterBuilder.build());
-        builder.addLimiter(bpsSoftLimiterBuilder.build());
-        builder.addLimiter(bpsHardLimiterBuilder.build());
-        builder.statsLogger(limiterStatLogger);
-        return builder.build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java
deleted file mode 100644
index c666b08..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/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.
- */
-/**
- * Request Rate Limiting.
- */
-package org.apache.distributedlog.service.stream.limiter;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/package-info.java
deleted file mode 100644
index 7429a85..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/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 Related Operations.
- */
-package org.apache.distributedlog.service.stream;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
deleted file mode 100644
index 72668c2..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.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 org.apache.distributedlog.service.streamset;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * A stream-to-partition converter that caches the mapping between stream and partitions.
- */
-public abstract class CacheableStreamPartitionConverter implements StreamPartitionConverter {
-
-    private final ConcurrentMap<String, Partition> partitions;
-
-    protected CacheableStreamPartitionConverter() {
-        this.partitions = new ConcurrentHashMap<String, Partition>();
-    }
-
-    @Override
-    public Partition convert(String streamName) {
-        Partition p = partitions.get(streamName);
-        if (null != p) {
-            return p;
-        }
-        // not found
-        Partition newPartition = newPartition(streamName);
-        Partition oldPartition = partitions.putIfAbsent(streamName, newPartition);
-        if (null == oldPartition) {
-            return newPartition;
-        } else {
-            return oldPartition;
-        }
-    }
-
-    /**
-     * Create the partition from <code>streamName</code>.
-     *
-     * @param streamName
-     *          stream name
-     * @return partition id of the stream
-     */
-    protected abstract Partition newPartition(String streamName);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
deleted file mode 100644
index 30b2896..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.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 org.apache.distributedlog.service.streamset;
-
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * Stream Partition Converter that converts the stream name into a stream-to-partition mapping by delimiter.
- */
-public class DelimiterStreamPartitionConverter extends CacheableStreamPartitionConverter {
-
-    private final String delimiter;
-
-    public DelimiterStreamPartitionConverter() {
-        this("_");
-    }
-
-    public DelimiterStreamPartitionConverter(String delimiter) {
-        this.delimiter = delimiter;
-    }
-
-    @Override
-    protected Partition newPartition(String streamName) {
-        String[] parts = StringUtils.split(streamName, delimiter);
-        if (null != parts && parts.length == 2) {
-            try {
-                int partition = Integer.parseInt(parts[1]);
-                return new Partition(parts[0], partition);
-            } catch (NumberFormatException nfe) {
-                // ignore the exception
-            }
-        }
-        return new Partition(streamName, 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
deleted file mode 100644
index 5be172f..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.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 org.apache.distributedlog.service.streamset;
-
-/**
- * Map stream name to partition of the same name.
- */
-public class IdentityStreamPartitionConverter extends CacheableStreamPartitionConverter {
-    @Override
-    protected Partition newPartition(String streamName) {
-        return new Partition(streamName, 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/Partition.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
deleted file mode 100644
index aa69276..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.streamset;
-
-import com.google.common.base.Objects;
-
-/**
- * `Partition` defines the relationship between a `virtual` stream and a
- * physical DL stream.
- *
- * <p>A `virtual` stream could be partitioned into multiple partitions
- * and each partition is effectively a DL stream.
- */
-public class Partition {
-
-    // Name of its parent stream.
-    private final String stream;
-
-    // Unique id of the partition within the stream.
-    // It can be just simply an index id.
-    public final int id;
-
-    public Partition(String stream, int id) {
-        this.stream = stream;
-        this.id = id;
-    }
-
-    /**
-     * Get the `virtual` stream name.
-     *
-     * @return the stream name.
-     */
-    public String getStream() {
-        return stream;
-    }
-
-    /**
-     * Get the partition id of this partition.
-     *
-     * @return partition id
-     */
-    public int getId() {
-        return id;
-    }
-
-    /**
-     * Get the 6 digit 0 padded id of this partition as a String.
-     * @return partition id
-     */
-    public String getPaddedId() {
-        return String.format("%06d", getId());
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (!(o instanceof Partition)) {
-            return false;
-        }
-        Partition partition = (Partition) o;
-
-        return id == partition.id && Objects.equal(stream, partition.stream);
-    }
-
-    @Override
-    public int hashCode() {
-        int result = stream.hashCode();
-        result = 31 * result + id;
-        return result;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("Partition(")
-          .append(stream)
-          .append(", ")
-          .append(id)
-          .append(")");
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java
deleted file mode 100644
index bfcc5db..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.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 org.apache.distributedlog.service.streamset;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A mapping between a logical stream and a set of physical partitions.
- */
-public class PartitionMap {
-
-    private final Map<String, Set<Partition>> partitionMap;
-
-    public PartitionMap() {
-        partitionMap = new HashMap<String, Set<Partition>>();
-    }
-
-    public synchronized boolean addPartition(Partition partition, int maxPartitions) {
-        if (maxPartitions <= 0) {
-            return true;
-        }
-        Set<Partition> partitions = partitionMap.get(partition.getStream());
-        if (null == partitions) {
-            partitions = new HashSet<Partition>();
-            partitions.add(partition);
-            partitionMap.put(partition.getStream(), partitions);
-            return true;
-        }
-        if (partitions.contains(partition) || partitions.size() < maxPartitions) {
-            partitions.add(partition);
-            return true;
-        }
-        return false;
-    }
-
-    public synchronized boolean removePartition(Partition partition) {
-        Set<Partition> partitions = partitionMap.get(partition.getStream());
-        return null != partitions && partitions.remove(partition);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java
deleted file mode 100644
index 3ea1337..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.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 org.apache.distributedlog.service.streamset;
-
-/**
- * Map stream name to a partition.
- *
- * @see Partition
- */
-public interface StreamPartitionConverter {
-
-    /**
-     * Convert the stream name to partition.
-     *
-     * @param streamName
-     *          stream name
-     * @return partition
-     */
-    Partition convert(String streamName);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/package-info.java
deleted file mode 100644
index d185e88..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/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.
- */
-/**
- * StreamSet - A logical set of streams.
- */
-package org.apache.distributedlog.service.streamset;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
deleted file mode 100644
index 3934eb5..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
+++ /dev/null
@@ -1,350 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.tools;
-
-import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.distributedlog.client.serverset.DLZkServerSet;
-import org.apache.distributedlog.service.ClientUtils;
-import org.apache.distributedlog.service.DLSocketAddress;
-import org.apache.distributedlog.service.DistributedLogClient;
-import org.apache.distributedlog.service.DistributedLogClientBuilder;
-import org.apache.distributedlog.tools.Tool;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Tools to interact with proxies.
- */
-public class ProxyTool extends Tool {
-
-    private static final Logger logger = LoggerFactory.getLogger(ProxyTool.class);
-
-    /**
-     * Abstract Cluster level command.
-     */
-    protected abstract static class ClusterCommand extends OptsCommand {
-
-        protected Options options = new Options();
-        protected URI uri;
-        protected final List<String> streams = new ArrayList<String>();
-
-        protected ClusterCommand(String name, String description) {
-            super(name, description);
-            options.addOption("u", "uri", true, "DistributedLog URI");
-            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 '0-9', list '1,2,3' and name '143'");
-        }
-
-        @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;
-            }
-
-            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
-            logger.info("Created serverset for {}", uri);
-            try {
-                DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                        .name("proxy_tool")
-                        .clientId(ClientId$.MODULE$.apply("proxy_tool"))
-                        .maxRedirects(2)
-                        .serverSet(serverSet.getServerSet())
-                        .clientBuilder(ClientBuilder.get()
-                            .connectionTimeout(Duration.fromSeconds(2))
-                            .tcpConnectTimeout(Duration.fromSeconds(2))
-                            .requestTimeout(Duration.fromSeconds(10))
-                            .hostConnectionLimit(1)
-                            .hostConnectionCoresize(1)
-                            .keepAlive(true)
-                            .failFast(false))
-                        .build();
-                try {
-                    return runCmd(client);
-                } finally {
-                    client.close();
-                }
-            } finally {
-                serverSet.close();
-            }
-        }
-
-        protected abstract int runCmd(DistributedLogClient client) 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.");
-            }
-            this.uri = URI.create(cmdline.getOptionValue("u"));
-
-            // get stream names
-            String streamPrefix = cmdline.hasOption("r") ? cmdline.getOptionValue("r") : "";
-            String streamExpression = null;
-            if (cmdline.hasOption("e")) {
-                streamExpression = cmdline.getOptionValue("e");
-            }
-            if (null == streamPrefix || null == streamExpression) {
-                throw new ParseException("Please specify stream prefix & expression.");
-            }
-            // 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) {
-                        streams.add(streamPrefix + part);
-                    }
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid stream suffix list : " + streamExpression);
-                }
-            } else {
-                streams.add(streamPrefix + streamExpression);
-            }
-        }
-    }
-
-    /**
-     * Command to release ownership of a log stream.
-     */
-    static class ReleaseCommand extends ClusterCommand {
-
-        double rate = 100f;
-
-        ReleaseCommand() {
-            super("release", "Release Stream Ownerships");
-            options.addOption("t", "rate", true, "Rate to release streams");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("t")) {
-                rate = Double.parseDouble(cmdline.getOptionValue("t", "100"));
-            }
-        }
-
-        @Override
-        protected int runCmd(DistributedLogClient client) throws Exception {
-            RateLimiter rateLimiter = RateLimiter.create(rate);
-            for (String stream : streams) {
-                rateLimiter.acquire();
-                try {
-                    Await.result(client.release(stream));
-                    System.out.println("Release ownership of stream " + stream);
-                } catch (Exception e) {
-                    System.err.println("Failed to release ownership of stream " + stream);
-                    throw e;
-                }
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "release [options]";
-        }
-    }
-
-    /**
-     * Command to truncate a log stream.
-     */
-    static class TruncateCommand extends ClusterCommand {
-
-        DLSN dlsn = DLSN.InitialDLSN;
-
-        TruncateCommand() {
-            super("truncate", "Truncate streams until given dlsn.");
-            options.addOption("d", "dlsn", true, "DLSN to truncate until");
-        }
-
-        @Override
-        protected int runCmd(DistributedLogClient client) throws Exception {
-            System.out.println("Truncating streams : " + streams);
-            for (String stream : streams) {
-                boolean success = Await.result(client.truncate(stream, dlsn));
-                System.out.println("Truncate " + stream + " to " + dlsn + " : " + success);
-            }
-            return 0;
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("d")) {
-                throw new ParseException("No DLSN provided");
-            }
-            String[] dlsnStrs = cmdline.getOptionValue("d").split(",");
-            if (dlsnStrs.length != 3) {
-                throw new ParseException("Invalid DLSN : " + cmdline.getOptionValue("d"));
-            }
-            dlsn = new DLSN(Long.parseLong(dlsnStrs[0]), Long.parseLong(dlsnStrs[1]), Long.parseLong(dlsnStrs[2]));
-        }
-
-        @Override
-        protected String getUsage() {
-            return "truncate [options]";
-        }
-    }
-
-    /**
-     * Abstract command to operate on a single proxy server.
-     */
-    protected abstract static class ProxyCommand extends OptsCommand {
-
-        protected Options options = new Options();
-        protected InetSocketAddress address;
-
-        protected ProxyCommand(String name, String description) {
-            super(name, description);
-            options.addOption("H", "host", true, "Single Proxy Address");
-        }
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (!cmdline.hasOption("H")) {
-                throw new ParseException("No proxy address provided");
-            }
-            address = DLSocketAddress.parseSocketAddress(cmdline.getOptionValue("H"));
-        }
-
-        @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;
-            }
-
-            DistributedLogClientBuilder clientBuilder = DistributedLogClientBuilder.newBuilder()
-                    .name("proxy_tool")
-                    .clientId(ClientId$.MODULE$.apply("proxy_tool"))
-                    .maxRedirects(2)
-                    .host(address)
-                    .clientBuilder(ClientBuilder.get()
-                            .connectionTimeout(Duration.fromSeconds(2))
-                            .tcpConnectTimeout(Duration.fromSeconds(2))
-                            .requestTimeout(Duration.fromSeconds(10))
-                            .hostConnectionLimit(1)
-                            .hostConnectionCoresize(1)
-                            .keepAlive(true)
-                            .failFast(false));
-            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
-                    ClientUtils.buildClient(clientBuilder);
-            try {
-                return runCmd(clientPair);
-            } finally {
-                clientPair.getLeft().close();
-            }
-        }
-
-        protected abstract int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client) throws Exception;
-    }
-
-    /**
-     * Command to enable/disable accepting new streams.
-     */
-    static class AcceptNewStreamCommand extends ProxyCommand {
-
-        boolean enabled = false;
-
-        AcceptNewStreamCommand() {
-            super("accept-new-stream", "Enable/Disable accepting new streams for one proxy");
-            options.addOption("e", "enabled", true, "Enable/Disable accepting new streams");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("e")) {
-                throw new ParseException("No action 'enable/disable' provided");
-            }
-            enabled = Boolean.parseBoolean(cmdline.getOptionValue("e"));
-        }
-
-        @Override
-        protected int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client)
-                throws Exception {
-            Await.result(client.getRight().setAcceptNewStream(enabled));
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "accept-new-stream [options]";
-        }
-    }
-
-    public ProxyTool() {
-        super();
-        addCommand(new ReleaseCommand());
-        addCommand(new TruncateCommand());
-        addCommand(new AcceptNewStreamCommand());
-    }
-
-    @Override
-    protected String getName() {
-        return "proxy_tool";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/package-info.java
deleted file mode 100644
index 92d0a7d..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/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.
- */
-/**
- * Service related tools.
- */
-package org.apache.distributedlog.service.tools;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java
deleted file mode 100644
index 9ee93b4..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.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 org.apache.distributedlog.service.utils;
-
-import java.io.IOException;
-import java.net.InetAddress;
-
-/**
- * Utils that used by servers.
- */
-public class ServerUtils {
-
-  /**
-   * Retrieve the ledger allocator pool name.
-   *
-   * @param serverRegionId region id that that server is running
-   * @param shardId shard id of the server
-   * @param useHostname whether to use hostname as the ledger allocator pool name
-   * @return ledger allocator pool name
-   * @throws IOException
-   */
-    public static String getLedgerAllocatorPoolName(int serverRegionId,
-                                                    int shardId,
-                                                    boolean useHostname)
-        throws IOException {
-        if (useHostname) {
-            return String.format("allocator_%04d_%s", serverRegionId,
-                InetAddress.getLocalHost().getHostAddress());
-        } else {
-            return String.format("allocator_%04d_%010d", serverRegionId, shardId);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/package-info.java
deleted file mode 100644
index 99cf736..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/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.
- */
-/**
- * Utilities used by proxy servers.
- */
-package org.apache.distributedlog.service.utils;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/resources/config/server_decider.conf
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/resources/config/server_decider.conf b/distributedlog-service/src/main/resources/config/server_decider.conf
deleted file mode 100644
index d2fddf5..0000000
--- a/distributedlog-service/src/main/resources/config/server_decider.conf
+++ /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.
-# */
-
-region_stop_accept_new_stream=0
-disable_durability_enforcement=0
-disable_write_limit=0
-bkc.repp_disable_durability_enforcement=0
-bkc.disable_ensemble_change=0
-dl.disable_logsegment_rolling=0
-dl.disable_write_limit=0
-bkc.atla.disallow_bookie_placement=0
-bkc.atlb.disallow_bookie_placement=0
-bkc.smf1.disallow_bookie_placement=0
-service_rate_limit_disabled=0
-service_checksum_disabled=0
-service_global_limiter_disabled=0

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/resources/config/server_decider.yml
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/resources/config/server_decider.yml b/distributedlog-service/src/main/resources/config/server_decider.yml
deleted file mode 100644
index 7df24bb..0000000
--- a/distributedlog-service/src/main/resources/config/server_decider.yml
+++ /dev/null
@@ -1,44 +0,0 @@
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-region_stop_accept_new_stream:
-  default_availability: 0
-disable_durability_enforcement:
-  default_availability: 0
-disable_write_limit:
-  default_availability: 0
-bkc.repp_disable_durability_enforcement:
-  default_availability: 0
-bkc.disable_ensemble_change:
-  default_availability: 0
-dl.disable_logsegment_rolling:
-  default_availability: 0
-dl.disable_write_limit:
-  default_availability: 0
-bkc.atla.disallow_bookie_placement:
-  default_availability: 0
-bkc.atlb.disallow_bookie_placement:
-  default_availability: 0
-bkc.smf1.disallow_bookie_placement:
-  default_availability: 0
-service_rate_limit_disabled:
-  default_availability: 0
-service_checksum_disabled:
-  default_availability: 0
-service_global_limiter_disabled:
-  default_availability: 0

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/resources/findbugsExclude.xml b/distributedlog-service/src/main/resources/findbugsExclude.xml
deleted file mode 100644
index e101a4d..0000000
--- a/distributedlog-service/src/main/resources/findbugsExclude.xml
+++ /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.
-//-->
-<FindBugsFilter>
-  <Match>
-    <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~org\.apache\.distributedlog\.thrift.*" />
-  </Match>
-  <Match>
-    <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~org\.apache\.distributedlog\.service\.placement\.thrift.*" />
-  </Match>
-  <Match>
-    <!-- it is safe to cast exception here. //-->
-    <Class name="org.apache.distributedlog.service.DistributedLogServiceImpl$Stream$2" />
-    <Method name="onFailure" />
-    <Bug pattern="BC_UNCONFIRMED_CAST" />
-  </Match>
-  <Match>
-    <!-- it is safe to cast exception here. //-->
-    <Class name="org.apache.distributedlog.service.stream.BulkWriteOp" />
-    <Method name="isDefiniteFailure" />
-    <Bug pattern="BC_IMPOSSIBLE_INSTANCEOF" />
-  </Match>
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/thrift/metadata.thrift
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/thrift/metadata.thrift b/distributedlog-service/src/main/thrift/metadata.thrift
deleted file mode 100644
index 9cb3c72..0000000
--- a/distributedlog-service/src/main/thrift/metadata.thrift
+++ /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.
- */
-namespace java org.apache.distributedlog.service.placement.thrift
-
-struct StreamLoad {
-    1: optional string stream
-    2: optional i32 load
-}
-
-struct ServerLoad {
-    1: optional string server
-    2: optional i64 load
-    3: optional list<StreamLoad> streams
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java b/distributedlog-service/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java
deleted file mode 100644
index a9ddae5..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.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 org.apache.distributedlog.client.routing;
-
-import com.google.common.collect.Sets;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-/**
- * A local routing service that used for testing.
- */
-public class LocalRoutingService implements RoutingService {
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder to build a local routing service for testing.
-     */
-    public static class Builder implements RoutingService.Builder {
-
-        private Builder() {}
-
-        @Override
-        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
-            return this;
-        }
-
-        @Override
-        public LocalRoutingService build() {
-            return new LocalRoutingService();
-        }
-    }
-
-    private final Map<String, LinkedHashSet<SocketAddress>> localAddresses =
-            new HashMap<String, LinkedHashSet<SocketAddress>>();
-    private final CopyOnWriteArrayList<RoutingListener> listeners =
-            new CopyOnWriteArrayList<RoutingListener>();
-
-    boolean allowRetrySameHost = true;
-
-    @Override
-    public void startService() {
-        // nop
-    }
-
-    @Override
-    public void stopService() {
-        // nop
-    }
-
-    @Override
-    public synchronized Set<SocketAddress> getHosts() {
-        Set<SocketAddress> hosts = Sets.newHashSet();
-        for (LinkedHashSet<SocketAddress> addresses : localAddresses.values()) {
-            hosts.addAll(addresses);
-        }
-        return hosts;
-    }
-
-    @Override
-    public RoutingService registerListener(RoutingListener listener) {
-        listeners.add(listener);
-        return this;
-    }
-
-    @Override
-    public RoutingService unregisterListener(RoutingListener listener) {
-        listeners.remove(listener);
-        return this;
-    }
-
-    public LocalRoutingService setAllowRetrySameHost(boolean enabled) {
-        allowRetrySameHost = enabled;
-        return this;
-    }
-
-    public LocalRoutingService addHost(String stream, SocketAddress address) {
-        boolean notify = false;
-        synchronized (this) {
-            LinkedHashSet<SocketAddress> addresses = localAddresses.get(stream);
-            if (null == addresses) {
-                addresses = new LinkedHashSet<SocketAddress>();
-                localAddresses.put(stream, addresses);
-            }
-            if (addresses.add(address)) {
-                notify = true;
-            }
-        }
-        if (notify) {
-            for (RoutingListener listener : listeners) {
-                listener.onServerJoin(address);
-            }
-        }
-        return this;
-    }
-
-    @Override
-    public synchronized SocketAddress getHost(String key, RoutingContext rContext)
-            throws NoBrokersAvailableException {
-        LinkedHashSet<SocketAddress> addresses = localAddresses.get(key);
-
-        SocketAddress candidate = null;
-        if (null != addresses) {
-            for (SocketAddress host : addresses) {
-                if (rContext.isTriedHost(host) && !allowRetrySameHost) {
-                    continue;
-                } else {
-                    candidate = host;
-                    break;
-                }
-            }
-        }
-        if (null != candidate) {
-            return candidate;
-        }
-        throw new NoBrokersAvailableException("No host available");
-    }
-
-    @Override
-    public void removeHost(SocketAddress address, Throwable reason) {
-        // nop
-    }
-}


[12/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
new file mode 100644
index 0000000..58b5b2a
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
@@ -0,0 +1,298 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.client.DistributedLogClientImpl;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.routing.LocalRoutingService;
+import org.apache.distributedlog.client.routing.RegionsRoutingService;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamManagerImpl;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration;
+import java.net.SocketAddress;
+import java.net.URI;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+/**
+ * Base test case for distributedlog servers.
+ */
+public abstract class DistributedLogServerTestCase {
+
+    protected static DistributedLogConfiguration conf =
+            new DistributedLogConfiguration().setLockTimeout(10)
+                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
+    protected static DistributedLogConfiguration noAdHocConf =
+            new DistributedLogConfiguration().setLockTimeout(10).setCreateStreamIfNotExists(false)
+                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
+    protected static DistributedLogCluster dlCluster;
+    protected static DistributedLogCluster noAdHocCluster;
+
+    /**
+     * A distributedlog client wrapper for testing.
+     */
+    protected static class DLClient {
+        public final LocalRoutingService routingService;
+        public DistributedLogClientBuilder dlClientBuilder;
+        public final DistributedLogClientImpl dlClient;
+
+        protected DLClient(String name,
+                           String streamNameRegex,
+                           Optional<String> serverSideRoutingFinagleName) {
+            routingService = LocalRoutingService.newBuilder().build();
+            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
+                        .name(name)
+                        .clientId(ClientId$.MODULE$.apply(name))
+                        .routingService(routingService)
+                        .streamNameRegex(streamNameRegex)
+                        .handshakeWithClientInfo(true)
+                        .clientBuilder(ClientBuilder.get()
+                            .hostConnectionLimit(1)
+                            .connectionTimeout(Duration.fromSeconds(1))
+                            .requestTimeout(Duration.fromSeconds(60)));
+            if (serverSideRoutingFinagleName.isPresent()) {
+                dlClientBuilder =
+                        dlClientBuilder.serverRoutingServiceFinagleNameStr(serverSideRoutingFinagleName.get());
+            }
+            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
+        }
+
+        public void handshake() {
+            dlClient.handshake();
+        }
+
+        public void shutdown() {
+            dlClient.close();
+        }
+    }
+
+    /**
+     * A distributedlog client wrapper that talks to two regions.
+     */
+    protected static class TwoRegionDLClient {
+
+        public final LocalRoutingService localRoutingService;
+        public final LocalRoutingService remoteRoutingService;
+        public final DistributedLogClientBuilder dlClientBuilder;
+        public final DistributedLogClientImpl dlClient;
+
+        protected TwoRegionDLClient(String name, Map<SocketAddress, String> regionMap) {
+            localRoutingService = new LocalRoutingService();
+            remoteRoutingService = new LocalRoutingService();
+            RegionsRoutingService regionsRoutingService =
+                    RegionsRoutingService.of(new DefaultRegionResolver(regionMap),
+                            localRoutingService, remoteRoutingService);
+            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
+                        .name(name)
+                        .clientId(ClientId$.MODULE$.apply(name))
+                        .routingService(regionsRoutingService)
+                        .streamNameRegex(".*")
+                        .handshakeWithClientInfo(true)
+                        .maxRedirects(2)
+                        .clientBuilder(ClientBuilder.get()
+                            .hostConnectionLimit(1)
+                            .connectionTimeout(Duration.fromSeconds(1))
+                            .requestTimeout(Duration.fromSeconds(10)));
+            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
+        }
+
+        public void shutdown() {
+            dlClient.close();
+        }
+    }
+
+    private final boolean clientSideRouting;
+    protected DLServer dlServer;
+    protected DLClient dlClient;
+    protected DLServer noAdHocServer;
+    protected DLClient noAdHocClient;
+
+    public static DistributedLogCluster createCluster(DistributedLogConfiguration conf) throws Exception {
+        return DistributedLogCluster.newBuilder()
+            .numBookies(3)
+            .shouldStartZK(true)
+            .zkServers("127.0.0.1")
+            .shouldStartProxy(false)
+            .dlConf(conf)
+            .bkConf(DLMTestUtil.loadTestBkConf())
+            .build();
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws Exception {
+        dlCluster = createCluster(conf);
+        dlCluster.start();
+    }
+
+    public void setupNoAdHocCluster() throws Exception {
+        noAdHocCluster = createCluster(noAdHocConf);
+        noAdHocCluster.start();
+        noAdHocServer = new DLServer(noAdHocConf, noAdHocCluster.getUri(), 7002, false);
+        Optional<String> serverSideRoutingFinagleName = Optional.absent();
+        if (!clientSideRouting) {
+            serverSideRoutingFinagleName =
+                    Optional.of("inet!" + DLSocketAddress.toString(noAdHocServer.getAddress()));
+        }
+        noAdHocClient = createDistributedLogClient("no-ad-hoc-client", serverSideRoutingFinagleName);
+    }
+
+    public void tearDownNoAdHocCluster() throws Exception {
+        if (null != noAdHocClient) {
+            noAdHocClient.shutdown();
+        }
+        if (null != noAdHocServer) {
+            noAdHocServer.shutdown();
+        }
+    }
+
+    @AfterClass
+    public static void teardownCluster() throws Exception {
+        if (null != dlCluster) {
+            dlCluster.stop();
+        }
+        if (null != noAdHocCluster) {
+            noAdHocCluster.stop();
+        }
+    }
+
+    protected static URI getUri() {
+        return dlCluster.getUri();
+    }
+
+    protected DistributedLogServerTestCase(boolean clientSideRouting) {
+        this.clientSideRouting = clientSideRouting;
+    }
+
+    @Before
+    public void setup() throws Exception {
+        dlServer = createDistributedLogServer(7001);
+        Optional<String> serverSideRoutingFinagleName = Optional.absent();
+        if (!clientSideRouting) {
+            serverSideRoutingFinagleName =
+                    Optional.of("inet!" + DLSocketAddress.toString(dlServer.getAddress()));
+        }
+        dlClient = createDistributedLogClient("test", serverSideRoutingFinagleName);
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != dlClient) {
+            dlClient.shutdown();
+        }
+        if (null != dlServer) {
+            dlServer.shutdown();
+        }
+    }
+
+    protected DLServer createDistributedLogServer(int port) throws Exception {
+        return new DLServer(conf, dlCluster.getUri(), port, false);
+    }
+
+    protected DLServer createDistributedLogServer(DistributedLogConfiguration conf, int port)
+            throws Exception {
+        return new DLServer(conf, dlCluster.getUri(), port, false);
+    }
+
+    protected DLClient createDistributedLogClient(String clientName,
+                                                  Optional<String> serverSideRoutingFinagleName)
+            throws Exception {
+        return createDistributedLogClient(clientName, ".*", serverSideRoutingFinagleName);
+    }
+
+    protected DLClient createDistributedLogClient(String clientName,
+                                                  String streamNameRegex,
+                                                  Optional<String> serverSideRoutingFinagleName)
+            throws Exception {
+        return new DLClient(clientName, streamNameRegex, serverSideRoutingFinagleName);
+    }
+
+    protected TwoRegionDLClient createTwoRegionDLClient(String clientName,
+                                                        Map<SocketAddress, String> regionMap)
+            throws Exception {
+        return new TwoRegionDLClient(clientName, regionMap);
+    }
+
+    protected static void checkStreams(int numExpectedStreams, DLServer dlServer) {
+        StreamManager streamManager = dlServer.dlServer.getKey().getStreamManager();
+        assertEquals(numExpectedStreams, streamManager.numCached());
+        assertEquals(numExpectedStreams, streamManager.numAcquired());
+    }
+
+    protected static void checkStreams(Set<String> streams, DLServer dlServer) {
+        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
+        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
+        Set<String> acquiredStreams = streamManager.getAcquiredStreams().keySet();
+
+        assertEquals(streams.size(), cachedStreams.size());
+        assertEquals(streams.size(), acquiredStreams.size());
+        assertTrue(Sets.difference(streams, cachedStreams).isEmpty());
+        assertTrue(Sets.difference(streams, acquiredStreams).isEmpty());
+    }
+
+    protected static void checkStream(String name, DLClient dlClient, DLServer dlServer,
+                                      int expectedNumProxiesInClient, int expectedClientCacheSize,
+                                      int expectedServerCacheSize, boolean existedInServer, boolean existedInClient) {
+        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
+        assertEquals(expectedNumProxiesInClient, distribution.size());
+
+        if (expectedNumProxiesInClient > 0) {
+            Map.Entry<SocketAddress, Set<String>> localEntry =
+                    distribution.entrySet().iterator().next();
+            assertEquals(dlServer.getAddress(), localEntry.getKey());
+            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
+            assertEquals(existedInClient, localEntry.getValue().contains(name));
+        }
+
+        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
+        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
+        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
+
+        assertEquals(expectedServerCacheSize, cachedStreams.size());
+        assertEquals(existedInServer, cachedStreams.contains(name));
+        assertEquals(expectedServerCacheSize, acquiredStreams.size());
+        assertEquals(existedInServer, acquiredStreams.contains(name));
+    }
+
+    protected static Map<SocketAddress, Set<String>> getStreamOwnershipDistribution(DLClient dlClient) {
+        return dlClient.dlClient.getStreamOwnershipDistribution();
+    }
+
+    protected static Set<String> getAllStreamsFromDistribution(Map<SocketAddress, Set<String>> distribution) {
+        Set<String> allStreams = new HashSet<String>();
+        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
+            allStreams.addAll(entry.getValue());
+        }
+        return allStreams;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
new file mode 100644
index 0000000..4a5dd01
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
@@ -0,0 +1,720 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.Charsets.UTF_8;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+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.base.Optional;
+import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogReader;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.client.routing.LocalRoutingService;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.impl.acl.ZKAccessControl;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.service.stream.StreamManagerImpl;
+import org.apache.distributedlog.thrift.AccessControlEntry;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteContext;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import com.twitter.util.Futures;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link DistributedLogServer}.
+ */
+public abstract class TestDistributedLogServerBase extends DistributedLogServerTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogServerBase.class);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    protected TestDistributedLogServerBase(boolean clientSideRouting) {
+        super(clientSideRouting);
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-27}.
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    public void testBasicWrite() throws Exception {
+        String name = "dlserver-basic-write";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        for (long i = 1; i <= 10; i++) {
+            logger.debug("Write entry {} to stream {}.", i, name);
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())));
+        }
+
+        HeartbeatOptions hbOptions = new HeartbeatOptions();
+        hbOptions.setSendHeartBeatToReader(true);
+        // make sure the first log segment of each stream created
+        FutureUtils.result(dlClient.dlClient.heartbeat(name));
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(1);
+        int numRead = 0;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            ++numRead;
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead, i);
+            r = reader.readNext(false);
+        }
+        assertEquals(10, numRead);
+        reader.close();
+        dlm.close();
+    }
+
+    /**
+     * Sanity check to make sure both checksum flag values work.
+     */
+    @Test(timeout = 60000)
+    public void testChecksumFlag() throws Exception {
+        String name = "testChecksumFlag";
+        LocalRoutingService routingService = LocalRoutingService.newBuilder().build();
+        routingService.addHost(name, dlServer.getAddress());
+        DistributedLogClientBuilder dlClientBuilder = DistributedLogClientBuilder.newBuilder()
+            .name(name)
+            .clientId(ClientId$.MODULE$.apply("test"))
+            .routingService(routingService)
+            .handshakeWithClientInfo(true)
+            .clientBuilder(ClientBuilder.get()
+                .hostConnectionLimit(1)
+                .connectionTimeout(Duration.fromSeconds(1))
+                .requestTimeout(Duration.fromSeconds(60)))
+            .checksum(false);
+        DistributedLogClient dlClient = dlClientBuilder.build();
+        Await.result(dlClient.write(name, ByteBuffer.wrap(("1").getBytes())));
+        dlClient.close();
+
+        dlClient = dlClientBuilder.checksum(true).build();
+        Await.result(dlClient.write(name, ByteBuffer.wrap(("2").getBytes())));
+        dlClient.close();
+    }
+
+    private void runSimpleBulkWriteTest(int writeCount) throws Exception {
+        String name = String.format("dlserver-bulk-write-%d", writeCount);
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+
+        logger.debug("Write {} entries to stream {}.", writeCount, name);
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        assertEquals(futures.size(), writeCount);
+        for (Future<DLSN> future : futures) {
+            // No throw == pass.
+            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
+        }
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(1);
+        int numRead = 0;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead + 1, i);
+            ++numRead;
+            r = reader.readNext(false);
+        }
+        assertEquals(writeCount, numRead);
+        reader.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWrite() throws Exception {
+        runSimpleBulkWriteTest(100);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteSingleWrite() throws Exception {
+        runSimpleBulkWriteTest(1);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteEmptyList() throws Exception {
+        String name = String.format("dlserver-bulk-write-%d", 0);
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+
+        assertEquals(0, futures.size());
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteNullArg() throws Exception {
+
+        String name = String.format("dlserver-bulk-write-%s", "null");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
+        writes.add(null);
+
+        try {
+            dlClient.dlClient.writeBulk(name, writes);
+            fail("should not have succeeded");
+        } catch (NullPointerException npe) {
+            // expected
+            logger.info("Expected to catch NullPointException.");
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteEmptyBuffer() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "empty");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
+        writes.add(ByteBuffer.wrap(("").getBytes()));
+        writes.add(ByteBuffer.wrap(("").getBytes()));
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        assertEquals(2, futures.size());
+        for (Future<DLSN> future : futures) {
+            // No throw == pass
+            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
+        }
+    }
+
+    void failDueToWrongException(Exception ex) {
+        logger.info("testBulkWritePartialFailure: ", ex);
+        fail(String.format("failed with wrong exception %s", ex.getClass().getName()));
+    }
+
+    int validateAllFailedAsCancelled(List<Future<DLSN>> futures, int start, int finish) {
+        int failed = 0;
+        for (int i = start; i < finish; i++) {
+            Future<DLSN> future = futures.get(i);
+            try {
+                Await.result(future, Duration.fromSeconds(10));
+                fail("future should have failed!");
+            } catch (DLException cre) {
+                ++failed;
+            } catch (Exception ex) {
+                failDueToWrongException(ex);
+            }
+        }
+        return failed;
+    }
+
+    void validateFailedAsLogRecordTooLong(Future<DLSN> future) {
+        try {
+            Await.result(future, Duration.fromSeconds(10));
+            fail("should have failed");
+        } catch (DLException dle) {
+            assertEquals(StatusCode.TOO_LARGE_RECORD.getValue(), dle.getCode());
+        } catch (Exception ex) {
+            failDueToWrongException(ex);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWritePartialFailure() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "partial-failure");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        final int writeCount = 100;
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount * 2 + 1);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+        // Too big, will cause partial failure.
+        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
+        writes.add(buf);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+
+        // Count succeeded.
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        int succeeded = 0;
+        for (int i = 0; i < writeCount; i++) {
+            Future<DLSN> future = futures.get(i);
+            try {
+                Await.result(future, Duration.fromSeconds(10));
+                ++succeeded;
+            } catch (Exception ex) {
+                failDueToWrongException(ex);
+            }
+        }
+
+        validateFailedAsLogRecordTooLong(futures.get(writeCount));
+        FutureUtils.result(Futures.collect(futures.subList(writeCount + 1, 2 * writeCount + 1)));
+        assertEquals(writeCount, succeeded);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteTotalFailureFirstWriteFailed() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "first-write-failed");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        final int writeCount = 100;
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
+        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
+        writes.add(buf);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        validateFailedAsLogRecordTooLong(futures.get(0));
+        FutureUtils.result(Futures.collect(futures.subList(1, writeCount + 1)));
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteTotalFailureLostLock() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "lost-lock");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        final int writeCount = 8;
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
+        ByteBuffer buf = ByteBuffer.allocate(8);
+        writes.add(buf);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+        // Warm it up with a write.
+        Await.result(dlClient.dlClient.write(name, ByteBuffer.allocate(8)));
+
+        // Failpoint a lost lock, make sure the failure gets promoted to an operation failure.
+        DistributedLogServiceImpl svcImpl = (DistributedLogServiceImpl) dlServer.dlServer.getLeft();
+        try {
+            FailpointUtils.setFailpoint(
+                FailpointUtils.FailPointName.FP_WriteInternalLostLock,
+                FailpointUtils.FailPointActions.FailPointAction_Default
+            );
+            Future<BulkWriteResponse> futures = svcImpl.writeBulkWithContext(name, writes, new WriteContext());
+            assertEquals(StatusCode.LOCKING_EXCEPTION, Await.result(futures).header.code);
+        } finally {
+            FailpointUtils.removeFailpoint(
+                FailpointUtils.FailPointName.FP_WriteInternalLostLock
+            );
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testHeartbeat() throws Exception {
+        String name = "dlserver-heartbeat";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        for (long i = 1; i <= 10; i++) {
+            logger.debug("Send heartbeat {} to stream {}.", i, name);
+            dlClient.dlClient.check(name).get();
+        }
+
+        logger.debug("Write entry one to stream {}.", name);
+        dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes())).get();
+
+        Thread.sleep(1000);
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
+        int numRead = 0;
+        // eid=0 => control records
+        // other 9 heartbeats will not trigger writing any control records.
+        // eid=1 => user entry
+        long startEntryId = 1;
+        LogRecordWithDLSN r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead + 1, i);
+            assertEquals(r.getDlsn().compareTo(new DLSN(1, startEntryId, 0)), 0);
+            ++numRead;
+            ++startEntryId;
+            r = reader.readNext(false);
+        }
+        assertEquals(1, numRead);
+    }
+
+    @Test(timeout = 60000)
+    public void testFenceWrite() throws Exception {
+        String name = "dlserver-fence-write";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        for (long i = 1; i <= 10; i++) {
+            logger.debug("Write entry {} to stream {}.", i, name);
+            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
+        }
+
+        Thread.sleep(1000);
+
+        logger.info("Fencing stream {}.", name);
+        DLMTestUtil.fenceStream(conf, getUri(), name);
+        logger.info("Fenced stream {}.", name);
+
+        for (long i = 11; i <= 20; i++) {
+            logger.debug("Write entry {} to stream {}.", i, name);
+            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
+        }
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(1);
+        int numRead = 0;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead + 1, i);
+            ++numRead;
+            r = reader.readNext(false);
+        }
+        assertEquals(20, numRead);
+        reader.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testDeleteStream() throws Exception {
+        String name = "dlserver-delete-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        long txid = 101;
+        for (long i = 1; i <= 10; i++) {
+            long curTxId = txid++;
+            logger.debug("Write entry {} to stream {}.", curTxId, name);
+            dlClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+        }
+
+        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
+
+        dlClient.dlClient.delete(name).get();
+
+        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
+
+        Thread.sleep(1000);
+
+        DistributedLogManager dlm101 = DLMTestUtil.createNewDLM(name, conf, getUri());
+        AsyncLogReader reader101 = FutureUtils.result(dlm101.openAsyncLogReader(DLSN.InitialDLSN));
+        try {
+            FutureUtils.result(reader101.readNext());
+            fail("Should fail with LogNotFoundException since the stream is deleted");
+        } catch (LogNotFoundException lnfe) {
+            // expected
+        }
+        FutureUtils.result(reader101.asyncClose());
+        dlm101.close();
+
+        txid = 201;
+        for (long i = 1; i <= 10; i++) {
+            long curTxId = txid++;
+            logger.debug("Write entry {} to stream {}.", curTxId, name);
+            DLSN dlsn = dlClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+        }
+        Thread.sleep(1000);
+
+        DistributedLogManager dlm201 = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader201 = dlm201.getInputStream(1);
+        int numRead = 0;
+        int curTxId = 201;
+        LogRecord r = reader201.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(curTxId++, i);
+            ++numRead;
+            r = reader201.readNext(false);
+        }
+        assertEquals(10, numRead);
+        reader201.close();
+        dlm201.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateStream() throws Exception {
+        try {
+            setupNoAdHocCluster();
+            final String name = "dlserver-create-stream";
+
+            noAdHocClient.routingService.addHost("dlserver-create-stream", noAdHocServer.getAddress());
+            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
+
+            long txid = 101;
+            for (long i = 1; i <= 10; i++) {
+                long curTxId = txid++;
+                logger.debug("Write entry {} to stream {}.", curTxId, name);
+                noAdHocClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+            }
+
+            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+        } finally {
+            tearDownNoAdHocCluster();
+        }
+    }
+
+    /**
+     * This tests that create has touch like behavior in that trying to create the stream twice, simply does nothing.
+     */
+    @Test(timeout = 60000)
+    public void testCreateStreamTwice() throws Exception {
+        try {
+            setupNoAdHocCluster();
+            final String name = "dlserver-create-stream-twice";
+
+            noAdHocClient.routingService.addHost("dlserver-create-stream-twice", noAdHocServer.getAddress());
+            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
+
+            long txid = 101;
+            for (long i = 1; i <= 10; i++) {
+                long curTxId = txid++;
+                logger.debug("Write entry {} to stream {}.", curTxId, name);
+                noAdHocClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+            }
+
+            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+
+            // create again
+            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
+            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+        } finally {
+            tearDownNoAdHocCluster();
+        }
+    }
+
+
+
+    @Test(timeout = 60000)
+    public void testTruncateStream() throws Exception {
+        String name = "dlserver-truncate-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        long txid = 1;
+        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
+        for (int s = 1; s <= 2; s++) {
+            for (long i = 1; i <= 10; i++) {
+                long curTxId = txid++;
+                logger.debug("Write entry {} to stream {}.", curTxId, name);
+                DLSN dlsn = dlClient.dlClient.write(name,
+                        ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+                txid2DLSN.put(curTxId, dlsn);
+            }
+            if (s == 1) {
+                dlClient.dlClient.release(name).get();
+            }
+        }
+
+        DLSN dlsnToDelete = txid2DLSN.get(11L);
+        dlClient.dlClient.truncate(name, dlsnToDelete).get();
+
+        DistributedLogManager readDLM = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = readDLM.getInputStream(1);
+        int numRead = 0;
+        int curTxId = 11;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(curTxId++, i);
+            ++numRead;
+            r = reader.readNext(false);
+        }
+        assertEquals(10, numRead);
+        reader.close();
+        readDLM.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testRequestDenied() throws Exception {
+        String name = "request-denied";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        AccessControlEntry ace = new AccessControlEntry();
+        ace.setDenyWrite(true);
+        ZooKeeperClient zkc = TestZooKeeperClientBuilder
+                .newBuilder()
+                .uri(getUri())
+                .connectionTimeoutMs(60000)
+                .sessionTimeoutMs(60000)
+                .build();
+        DistributedLogNamespace dlNamespace = dlServer.dlServer.getLeft().getDistributedLogNamespace();
+        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
+        String zkPath = getUri().getPath() + "/" + bkdlConfig.getACLRootPath() + "/" + name;
+        ZKAccessControl accessControl = new ZKAccessControl(ace, zkPath);
+        accessControl.create(zkc);
+
+        AccessControlManager acm = dlNamespace.createAccessControlManager();
+        while (acm.allowWrite(name)) {
+            Thread.sleep(100);
+        }
+
+        try {
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+            fail("Should fail with request denied exception");
+        } catch (DLException dle) {
+            assertEquals(StatusCode.REQUEST_DENIED.getValue(), dle.getCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testNoneStreamNameRegex() throws Exception {
+        String streamNamePrefix = "none-stream-name-regex-";
+        int numStreams = 5;
+        Set<String> streams = new HashSet<String>();
+
+        for (int i = 0; i < numStreams; i++) {
+            streams.add(streamNamePrefix + i);
+        }
+        testStreamNameRegex(streams, ".*", streams);
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamNameRegex() throws Exception {
+        String streamNamePrefix = "stream-name-regex-";
+        int numStreams = 5;
+        Set<String> streams = new HashSet<String>();
+        Set<String> expectedStreams = new HashSet<String>();
+        String streamNameRegex = streamNamePrefix + "1";
+
+        for (int i = 0; i < numStreams; i++) {
+            streams.add(streamNamePrefix + i);
+        }
+        expectedStreams.add(streamNamePrefix + "1");
+
+        testStreamNameRegex(streams, streamNameRegex, expectedStreams);
+    }
+
+    private void testStreamNameRegex(Set<String> streams, String streamNameRegex,
+                                     Set<String> expectedStreams)
+            throws Exception {
+        for (String streamName : streams) {
+            dlClient.routingService.addHost(streamName, dlServer.getAddress());
+            Await.result(dlClient.dlClient.write(streamName,
+                    ByteBuffer.wrap(streamName.getBytes(UTF_8))));
+        }
+
+        DLClient client = createDistributedLogClient(
+                "test-stream-name-regex",
+                streamNameRegex,
+                Optional.<String>absent());
+        try {
+            client.routingService.addHost("unknown", dlServer.getAddress());
+            client.handshake();
+            Map<SocketAddress, Set<String>> distribution =
+                    client.dlClient.getStreamOwnershipDistribution();
+            assertEquals(1, distribution.size());
+            Set<String> cachedStreams = distribution.values().iterator().next();
+            assertNotNull(cachedStreams);
+            assertEquals(expectedStreams.size(), cachedStreams.size());
+
+            for (String streamName : cachedStreams) {
+                assertTrue(expectedStreams.contains(streamName));
+            }
+        } finally {
+            client.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testReleaseStream() throws Exception {
+        String name = "dlserver-release-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
+
+        // release the stream
+        Await.result(dlClient.dlClient.release(name));
+        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
+    }
+
+    protected void checkStream(int expectedNumProxiesInClient, int expectedClientCacheSize, int expectedServerCacheSize,
+                             String name, SocketAddress owner, boolean existedInServer, boolean existedInClient) {
+        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
+        assertEquals(expectedNumProxiesInClient, distribution.size());
+
+        if (expectedNumProxiesInClient > 0) {
+            Map.Entry<SocketAddress, Set<String>> localEntry =
+                    distribution.entrySet().iterator().next();
+            assertEquals(owner, localEntry.getKey());
+            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
+            assertEquals(existedInClient, localEntry.getValue().contains(name));
+        }
+
+
+        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
+        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
+        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
+
+        assertEquals(expectedServerCacheSize, cachedStreams.size());
+        assertEquals(existedInServer, cachedStreams.contains(name));
+        assertEquals(expectedServerCacheSize, acquiredStreams.size());
+        assertEquals(existedInServer, acquiredStreams.contains(name));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java
new file mode 100644
index 0000000..c7ae960
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.Charsets.UTF_8;
+import static org.junit.Assert.fail;
+
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import org.junit.Test;
+
+/**
+ * Test the server with client side routing.
+ */
+public class TestDistributedLogServerClientRouting extends TestDistributedLogServerBase {
+
+    public TestDistributedLogServerClientRouting() {
+        super(true);
+    }
+
+    @Test(timeout = 60000)
+    public void testAcceptNewStream() throws Exception {
+        String name = "dlserver-accept-new-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+        dlClient.routingService.setAllowRetrySameHost(false);
+
+        Await.result(dlClient.dlClient.setAcceptNewStream(false));
+
+        try {
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+            fail("Should fail because the proxy couldn't accept new stream");
+        } catch (NoBrokersAvailableException nbae) {
+            // expected
+        }
+        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
+
+        Await.result(dlServer.dlServer.getLeft().setAcceptNewStream(true));
+        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java
new file mode 100644
index 0000000..12416a3
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.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.service;
+
+/**
+ * Test the server with client side routing.
+ */
+public class TestDistributedLogServerServerRouting extends TestDistributedLogServerBase {
+
+    public TestDistributedLogServerServerRouting() {
+        super(false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
new file mode 100644
index 0000000..4a2d65f
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
@@ -0,0 +1,833 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.acl.DefaultAccessControlManager;
+import org.apache.distributedlog.client.routing.LocalRoutingService;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.protocol.util.ProtocolUtils;
+import org.apache.distributedlog.service.config.NullStreamConfigProvider;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
+import org.apache.distributedlog.service.stream.Stream;
+import org.apache.distributedlog.service.stream.StreamImpl;
+import org.apache.distributedlog.service.stream.StreamImpl.StreamStatus;
+import org.apache.distributedlog.service.stream.StreamManagerImpl;
+import org.apache.distributedlog.service.stream.WriteOp;
+import org.apache.distributedlog.service.streamset.DelimiterStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+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.ConfUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.ConfigurationException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for DistributedLog Service.
+ */
+public class TestDistributedLogService extends TestDistributedLogBase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogService.class);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private ServerConfiguration serverConf;
+    private DistributedLogConfiguration dlConf;
+    private URI uri;
+    private final CountDownLatch latch = new CountDownLatch(1);
+    private DistributedLogServiceImpl service;
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        dlConf = new DistributedLogConfiguration();
+        dlConf.addConfiguration(conf);
+        dlConf.setLockTimeout(0)
+                .setOutputBufferSize(0)
+                .setPeriodicFlushFrequencyMilliSeconds(10)
+                .setSchedulerShutdownTimeoutMs(100);
+        serverConf = newLocalServerConf();
+        uri = createDLMURI("/" + testName.getMethodName());
+        ensureURICreated(uri);
+        service = createService(serverConf, dlConf, latch);
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        if (null != service) {
+            service.shutdown();
+        }
+        super.teardown();
+    }
+
+    private DistributedLogConfiguration newLocalConf() {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(dlConf);
+        return confLocal;
+    }
+
+    private ServerConfiguration newLocalServerConf() {
+        ServerConfiguration serverConf = new ServerConfiguration();
+        serverConf.loadConf(dlConf);
+        serverConf.setServerThreads(1);
+        return serverConf;
+    }
+
+    private DistributedLogServiceImpl createService(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf) throws Exception {
+        return createService(serverConf, dlConf, new CountDownLatch(1));
+    }
+
+    private DistributedLogServiceImpl createService(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf,
+            CountDownLatch latch) throws Exception {
+        // Build the stream partition converter
+        StreamPartitionConverter converter;
+        try {
+            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
+        } catch (ConfigurationException e) {
+            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
+                    IdentityStreamPartitionConverter.class.getName());
+            converter = new IdentityStreamPartitionConverter();
+        }
+        return new DistributedLogServiceImpl(
+            serverConf,
+            dlConf,
+            ConfUtils.getConstDynConf(dlConf),
+            new NullStreamConfigProvider(),
+            uri,
+            converter,
+            new LocalRoutingService(),
+            NullStatsLogger.INSTANCE,
+            NullStatsLogger.INSTANCE,
+            latch,
+            new EqualLoadAppraiser());
+    }
+
+    private StreamImpl createUnstartedStream(DistributedLogServiceImpl service,
+                                             String name) throws Exception {
+        StreamImpl stream = (StreamImpl) service.newStream(name);
+        stream.initialize();
+        return stream;
+    }
+
+    private ByteBuffer createRecord(long txid) {
+        return ByteBuffer.wrap(("record-" + txid).getBytes(UTF_8));
+    }
+
+    private WriteOp createWriteOp(DistributedLogServiceImpl service,
+                                  String streamName,
+                                  long txid) {
+        ByteBuffer data = createRecord(txid);
+        return service.newWriteOp(streamName, data, null);
+    }
+
+    @Test(timeout = 60000)
+    public void testAcquireStreams() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s0 = createUnstartedStream(service, streamName);
+        ServerConfiguration serverConf1 = new ServerConfiguration();
+        serverConf1.addConfiguration(serverConf);
+        serverConf1.setServerPort(9999);
+        DistributedLogServiceImpl service1 = createService(serverConf1, dlConf);
+        StreamImpl s1 = createUnstartedStream(service1, streamName);
+
+        // create write ops
+        WriteOp op0 = createWriteOp(service, streamName, 0L);
+        s0.submit(op0);
+
+        WriteOp op1 = createWriteOp(service1, streamName, 1L);
+        s1.submit(op1);
+
+        // check pending size
+        assertEquals("Write Op 0 should be pending in service 0",
+                1, s0.numPendingOps());
+        assertEquals("Write Op 1 should be pending in service 1",
+                1, s1.numPendingOps());
+
+        // start acquiring s0
+        s0.start();
+        WriteResponse wr0 = Await.result(op0.result());
+        assertEquals("Op 0 should succeed",
+                StatusCode.SUCCESS, wr0.getHeader().getCode());
+        assertEquals("Service 0 should acquire stream",
+                StreamStatus.INITIALIZED, s0.getStatus());
+        assertNotNull(s0.getManager());
+        assertNotNull(s0.getWriter());
+        assertNull(s0.getLastException());
+
+        // start acquiring s1
+        s1.start();
+        WriteResponse wr1 = Await.result(op1.result());
+        assertEquals("Op 1 should fail",
+                StatusCode.FOUND, wr1.getHeader().getCode());
+        // the stream will be set to ERROR and then be closed.
+        assertTrue("Service 1 should be in unavailable state",
+                StreamStatus.isUnavailable(s1.getStatus()));
+        assertNotNull(s1.getManager());
+        assertNull(s1.getWriter());
+        assertNotNull(s1.getLastException());
+        assertTrue(s1.getLastException() instanceof OwnershipAcquireFailedException);
+
+        service1.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testAcquireStreamsWhenExceedMaxCachedPartitions() throws Exception {
+        String streamName = testName.getMethodName() + "_0000";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(dlConf);
+        confLocal.setMaxCachedPartitionsPerProxy(1);
+
+        ServerConfiguration serverConfLocal = new ServerConfiguration();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
+
+        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
+        Stream stream = serviceLocal.getLogWriter(streamName);
+
+        // stream is cached
+        assertNotNull(stream);
+        assertEquals(1, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op0 = createWriteOp(service, streamName, 0L);
+        stream.submit(op0);
+        WriteResponse wr0 = Await.result(op0.result());
+        assertEquals("Op 0 should succeed",
+                StatusCode.SUCCESS, wr0.getHeader().getCode());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+
+        // should fail to acquire another partition
+        try {
+            serviceLocal.getLogWriter(testName.getMethodName() + "_0001");
+            fail("Should fail to acquire new streams");
+        } catch (StreamUnavailableException sue) {
+            // expected
+        }
+        assertEquals(1, serviceLocal.getStreamManager().numCached());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+
+        // should be able to acquire partitions from other streams
+        String anotherStreamName = testName.getMethodName() + "-another_0001";
+        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
+        assertNotNull(anotherStream);
+        assertEquals(2, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
+        anotherStream.submit(op1);
+        WriteResponse wr1 = Await.result(op1.result());
+        assertEquals("Op 1 should succeed",
+                StatusCode.SUCCESS, wr1.getHeader().getCode());
+        assertEquals(2, serviceLocal.getStreamManager().numAcquired());
+    }
+
+    @Test(timeout = 60000)
+    public void testAcquireStreamsWhenExceedMaxAcquiredPartitions() throws Exception {
+        String streamName = testName.getMethodName() + "_0000";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(dlConf);
+        confLocal.setMaxCachedPartitionsPerProxy(-1);
+        confLocal.setMaxAcquiredPartitionsPerProxy(1);
+
+        ServerConfiguration serverConfLocal = new ServerConfiguration();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
+
+        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
+        Stream stream = serviceLocal.getLogWriter(streamName);
+
+        // stream is cached
+        assertNotNull(stream);
+        assertEquals(1, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op0 = createWriteOp(service, streamName, 0L);
+        stream.submit(op0);
+        WriteResponse wr0 = Await.result(op0.result());
+        assertEquals("Op 0 should succeed",
+                StatusCode.SUCCESS, wr0.getHeader().getCode());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+
+        // should be able to cache partitions from same stream
+        String anotherStreamName = testName.getMethodName() + "_0001";
+        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
+        assertNotNull(anotherStream);
+        assertEquals(2, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
+        anotherStream.submit(op1);
+        WriteResponse wr1 = Await.result(op1.result());
+        assertEquals("Op 1 should fail",
+                StatusCode.STREAM_UNAVAILABLE, wr1.getHeader().getCode());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseShouldErrorOutPendingOps() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s = createUnstartedStream(service, streamName);
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
+        for (int i = 0; i < numWrites; i++) {
+            WriteOp op = createWriteOp(service, streamName, i);
+            s.submit(op);
+            futureList.add(op.result());
+        }
+        assertEquals(numWrites, s.numPendingOps());
+        Await.result(s.requestClose("close stream"));
+        assertEquals("Stream " + streamName + " is set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+        for (int i = 0; i < numWrites; i++) {
+            Future<WriteResponse> future = futureList.get(i);
+            WriteResponse wr = Await.result(future);
+            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
+                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseTwice() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s = createUnstartedStream(service, streamName);
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
+        for (int i = 0; i < numWrites; i++) {
+            WriteOp op = createWriteOp(service, streamName, i);
+            s.submit(op);
+            futureList.add(op.result());
+        }
+        assertEquals(numWrites, s.numPendingOps());
+
+        Future<Void> closeFuture0 = s.requestClose("close 0");
+        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
+                StreamStatus.CLOSING == s.getStatus()
+                    || StreamStatus.CLOSED == s.getStatus());
+        Future<Void> closeFuture1 = s.requestClose("close 1");
+        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
+                StreamStatus.CLOSING == s.getStatus()
+                    || StreamStatus.CLOSED == s.getStatus());
+
+        Await.result(closeFuture0);
+        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+        Await.result(closeFuture1);
+        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+
+        for (int i = 0; i < numWrites; i++) {
+            Future<WriteResponse> future = futureList.get(i);
+            WriteResponse wr = Await.result(future);
+            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
+                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testFailRequestsDuringClosing() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s = createUnstartedStream(service, streamName);
+
+        Future<Void> closeFuture = s.requestClose("close");
+        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
+                StreamStatus.CLOSING == s.getStatus()
+                    || StreamStatus.CLOSED == s.getStatus());
+        WriteOp op1 = createWriteOp(service, streamName, 0L);
+        s.submit(op1);
+        WriteResponse response1 = Await.result(op1.result());
+        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closing",
+                StatusCode.STREAM_UNAVAILABLE, response1.getHeader().getCode());
+
+        Await.result(closeFuture);
+        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+        WriteOp op2 = createWriteOp(service, streamName, 1L);
+        s.submit(op2);
+        WriteResponse response2 = Await.result(op2.result());
+        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closed",
+                StatusCode.STREAM_UNAVAILABLE, response2.getHeader().getCode());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceTimeout() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+        ServerConfiguration serverConfLocal = newLocalServerConf();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.setServiceTimeoutMs(200)
+                .setStreamProbationTimeoutMs(100);
+        String streamName = testName.getMethodName();
+        // create a new service with 200ms timeout
+        DistributedLogServiceImpl localService = createService(serverConfLocal, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
+        for (int i = 0; i < numWrites; i++) {
+            futureList.add(localService.write(streamName, createRecord(i)));
+        }
+
+        assertTrue("Stream " + streamName + " should be cached",
+                streamManager.getCachedStreams().containsKey(streamName));
+
+        StreamImpl s = (StreamImpl) streamManager.getCachedStreams().get(streamName);
+        // the stream should be set CLOSING
+        while (StreamStatus.CLOSING != s.getStatus()
+            && StreamStatus.CLOSED != s.getStatus()) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+        assertNotNull("Writer should be initialized", s.getWriter());
+        assertNull("No exception should be thrown", s.getLastException());
+        Future<Void> closeFuture = s.getCloseFuture();
+        Await.result(closeFuture);
+        for (int i = 0; i < numWrites; i++) {
+            assertTrue("Write should not fail before closing",
+                    futureList.get(i).isDefined());
+            WriteResponse response = Await.result(futureList.get(i));
+            assertTrue("Op should fail with " + StatusCode.WRITE_CANCELLED_EXCEPTION,
+                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
+                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
+                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
+        }
+
+        while (streamManager.getCachedStreams().containsKey(streamName)) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        assertFalse("Stream should be removed from cache",
+                streamManager.getCachedStreams().containsKey(streamName));
+        assertFalse("Stream should be removed from acquired cache",
+                streamManager.getAcquiredStreams().containsKey(streamName));
+
+        localService.shutdown();
+    }
+
+    private DistributedLogServiceImpl createConfiguredLocalService() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+        return createService(serverConf, confLocal);
+    }
+
+    private ByteBuffer getTestDataBuffer() {
+        return ByteBuffer.wrap("test-data".getBytes());
+    }
+
+    @Test(timeout = 60000)
+    public void testNonDurableWrite() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0)
+                .setDurableWriteEnabled(false);
+        ServerConfiguration serverConfLocal = new ServerConfiguration();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.enableDurableWrite(false);
+        serverConfLocal.setServiceTimeoutMs(Integer.MAX_VALUE)
+                .setStreamProbationTimeoutMs(Integer.MAX_VALUE);
+        String streamName = testName.getMethodName();
+        DistributedLogServiceImpl localService =
+                createService(serverConfLocal, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>();
+        for (int i = 0; i < numWrites; i++) {
+            futureList.add(localService.write(streamName, createRecord(i)));
+        }
+        assertTrue("Stream " + streamName + " should be cached",
+                streamManager.getCachedStreams().containsKey(streamName));
+        List<WriteResponse> resultList = FutureUtils.result(Future.collect(futureList));
+        for (WriteResponse wr : resultList) {
+            assertEquals(DLSN.InvalidDLSN, DLSN.deserialize(wr.getDlsn()));
+        }
+
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpNoChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext();
+        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testTruncateOpNoChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext();
+        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpNoChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext();
+        HeartbeatOptions option = new HeartbeatOptions();
+        option.setSendHeartBeatToReader(true);
+
+        // hearbeat to acquire the stream and then release the stream
+        Future<WriteResponse> result = localService.heartbeatWithOptions("test", ctx, option);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        result = localService.release("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+
+        // heartbeat to acquire the stream and then delete the stream
+        result = localService.heartbeatWithOptions("test", ctx, option);
+        resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        result = localService.delete("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+
+        // shutdown the local service
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpChecksumBadChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(999);
+        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpChecksumBadStream() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(
+            ProtocolUtils.writeOpCRC32("test", getTestDataBuffer().array()));
+        Future<WriteResponse> result = localService.writeWithContext("test1", getTestDataBuffer(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpChecksumBadData() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        ByteBuffer buffer = getTestDataBuffer();
+        WriteContext ctx = new WriteContext().setCrc32(
+            ProtocolUtils.writeOpCRC32("test", buffer.array()));
+
+        // Overwrite 1 byte to corrupt data.
+        buffer.put(1, (byte) 0xab);
+        Future<WriteResponse> result = localService.writeWithContext("test", buffer, ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpChecksumBadChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(999);
+        Future<WriteResponse> result = localService.heartbeat("test", ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        result = localService.release("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        result = localService.delete("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testTruncateOpChecksumBadChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(999);
+        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    private WriteOp getWriteOp(String name, SettableFeature disabledFeature, Long checksum) {
+        return new WriteOp(name,
+            ByteBuffer.wrap("test".getBytes()),
+            new NullStatsLogger(),
+            new NullStatsLogger(),
+            new IdentityStreamPartitionConverter(),
+            new ServerConfiguration(),
+            (byte) 0,
+            checksum,
+            false,
+            disabledFeature,
+            DefaultAccessControlManager.INSTANCE);
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpBadChecksumWithChecksumDisabled() throws Exception {
+        String streamName = testName.getMethodName();
+
+        SettableFeature disabledFeature = new SettableFeature("", 0);
+
+        WriteOp writeOp0 = getWriteOp(streamName, disabledFeature, 919191L);
+        WriteOp writeOp1 = getWriteOp(streamName, disabledFeature, 919191L);
+
+        try {
+            writeOp0.preExecute();
+            fail("should have thrown");
+        } catch (Exception ex) {
+        }
+
+        disabledFeature.set(1);
+        writeOp1.preExecute();
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpGoodChecksumWithChecksumDisabled() throws Exception {
+        String streamName = testName.getMethodName();
+
+        SettableFeature disabledFeature = new SettableFeature("", 1);
+        WriteOp writeOp0 = getWriteOp(
+            streamName,
+            disabledFeature,
+            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
+        WriteOp writeOp1 = getWriteOp(
+            streamName,
+            disabledFeature,
+            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
+
+        writeOp0.preExecute();
+        disabledFeature.set(0);
+        writeOp1.preExecute();
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseStreamsShouldFlush() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+
+        String streamNamePrefix = testName.getMethodName();
+        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numStreams = 10;
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList =
+                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
+        for (int i = 0; i < numStreams; i++) {
+            String streamName = streamNamePrefix + "-" + i;
+            HeartbeatOptions hbOptions = new HeartbeatOptions();
+            hbOptions.setSendHeartBeatToReader(true);
+            // make sure the first log segment of each stream created
+            FutureUtils.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
+            for (int j = 0; j < numWrites; j++) {
+                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
+            }
+        }
+
+        assertEquals("There should be " + numStreams + " streams in cache",
+                numStreams, streamManager.getCachedStreams().size());
+        while (streamManager.getAcquiredStreams().size() < numStreams) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        Future<List<Void>> closeResult = localService.closeStreams();
+        List<Void> closedStreams = Await.result(closeResult);
+        assertEquals("There should be " + numStreams + " streams closed",
+                numStreams, closedStreams.size());
+        // all writes should be flushed
+        for (Future<WriteResponse> future : futureList) {
+            WriteResponse response = Await.result(future);
+            assertTrue("Op should succeed or be rejected : " + response.getHeader().getCode(),
+                    StatusCode.SUCCESS == response.getHeader().getCode()
+                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
+                        || StatusCode.STREAM_UNAVAILABLE == response.getHeader().getCode());
+        }
+        assertTrue("There should be no streams in the cache",
+                streamManager.getCachedStreams().isEmpty());
+        assertTrue("There should be no streams in the acquired cache",
+                streamManager.getAcquiredStreams().isEmpty());
+
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseStreamsShouldAbort() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+
+        String streamNamePrefix = testName.getMethodName();
+        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numStreams = 10;
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList =
+                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
+        for (int i = 0; i < numStreams; i++) {
+            String streamName = streamNamePrefix + "-" + i;
+            HeartbeatOptions hbOptions = new HeartbeatOptions();
+            hbOptions.setSendHeartBeatToReader(true);
+            // make sure the first log segment of each stream created
+            FutureUtils.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
+            for (int j = 0; j < numWrites; j++) {
+                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
+            }
+        }
+
+        assertEquals("There should be " + numStreams + " streams in cache",
+                numStreams, streamManager.getCachedStreams().size());
+        while (streamManager.getAcquiredStreams().size() < numStreams) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        for (Stream s : streamManager.getAcquiredStreams().values()) {
+            StreamImpl stream = (StreamImpl) s;
+            stream.setStatus(StreamStatus.ERROR);
+        }
+
+        Future<List<Void>> closeResult = localService.closeStreams();
+        List<Void> closedStreams = Await.result(closeResult);
+        assertEquals("There should be " + numStreams + " streams closed",
+                numStreams, closedStreams.size());
+        // all writes should be flushed
+        for (Future<WriteResponse> future : futureList) {
+            WriteResponse response = Await.result(future);
+            assertTrue("Op should fail with " + StatusCode.BK_TRANSMIT_ERROR + " or be rejected : "
+                    + response.getHeader().getCode(),
+                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
+                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
+                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
+        }
+        // acquired streams should all been removed after we close them
+        assertTrue("There should be no streams in the acquired cache",
+            streamManager.getAcquiredStreams().isEmpty());
+        localService.shutdown();
+        // cached streams wouldn't be removed immediately after streams are closed
+        // but they should be removed after we shutdown the service
+        assertTrue("There should be no streams in the cache after shutting down the service",
+            streamManager.getCachedStreams().isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testShutdown() throws Exception {
+        service.shutdown();
+        StreamManagerImpl streamManager = (StreamManagerImpl) service.getStreamManager();
+        WriteResponse response =
+                Await.result(service.write(testName.getMethodName(), createRecord(0L)));
+        assertEquals("Write should fail with " + StatusCode.SERVICE_UNAVAILABLE,
+                StatusCode.SERVICE_UNAVAILABLE, response.getHeader().getCode());
+        assertTrue("There should be no streams created after shutdown",
+                streamManager.getCachedStreams().isEmpty());
+        assertTrue("There should be no streams acquired after shutdown",
+                streamManager.getAcquiredStreams().isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetOwner() throws Exception {
+        ((LocalRoutingService) service.getRoutingService())
+                .addHost("stream-0", service.getServiceAddress().getSocketAddress())
+                .setAllowRetrySameHost(false);
+
+        service.startPlacementPolicy();
+
+        WriteResponse response = FutureUtils.result(service.getOwner("stream-1", new WriteContext()));
+        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
+        assertEquals(service.getServiceAddress().toString(),
+                response.getHeader().getLocation());
+
+        // service cache "stream-2"
+        StreamImpl stream = (StreamImpl) service.getStreamManager().getOrCreateStream("stream-2", false);
+        // create write ops to stream-2 to make service acquire the stream
+        WriteOp op = createWriteOp(service, "stream-2", 0L);
+        stream.submit(op);
+        stream.start();
+        WriteResponse wr = Await.result(op.result());
+        assertEquals("Op should succeed",
+                StatusCode.SUCCESS, wr.getHeader().getCode());
+        assertEquals("Service should acquire stream",
+                StreamStatus.INITIALIZED, stream.getStatus());
+        assertNotNull(stream.getManager());
+        assertNotNull(stream.getWriter());
+        assertNull(stream.getLastException());
+
+        // the stream is acquired
+        response = FutureUtils.result(service.getOwner("stream-2", new WriteContext()));
+        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
+        assertEquals(service.getServiceAddress().toString(),
+                response.getHeader().getLocation());
+    }
+
+}


[28/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 6ef1d8e..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 17b70be..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 0a6b076..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 4161afb..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.proxy;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 2ac5be3..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 023799c..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 81cda2f..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.resolver;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 666fa31..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index e51eb1e..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index d71cee3..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index ad73c17..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 4ac22ce..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 8e8edd3..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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);
-    }
-
-}


[24/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java
index e16fcab..50c151f 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * An exception is thrown when a log writer attempts to write a record with out-of-order transaction id.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java
index 637886e..50c21aa 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * An {@code UnexpectedException} is thrown when encountering unexpected conditions.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java
index 01fab89..7eb6ed5 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when found unsupported metadata version.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java
index d9001dd..8db1d50 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Signals when a write request is cancelled.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java
index 6899dbf..1d9c2a9 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * An exception on writing log records.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/ProtocolUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/ProtocolUtils.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/ProtocolUtils.java
deleted file mode 100644
index 0a37d3a..0000000
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/ProtocolUtils.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.util;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import org.apache.distributedlog.DLSN;
-import java.util.zip.CRC32;
-
-/**
- * With CRC embedded in the application, we have to keep track of per api crc. Ideally this
- * would be done by thrift.
- */
-public class ProtocolUtils {
-
-    // For request payload checksum
-    private static final ThreadLocal<CRC32> requestCRC = new ThreadLocal<CRC32>() {
-        @Override
-        protected CRC32 initialValue() {
-            return new CRC32();
-        }
-    };
-
-    /**
-     * Generate crc32 for WriteOp.
-     */
-    public static Long writeOpCRC32(String stream, byte[] payload) {
-        CRC32 crc = requestCRC.get();
-        try {
-            crc.update(stream.getBytes(UTF_8));
-            crc.update(payload);
-            return crc.getValue();
-        } finally {
-            crc.reset();
-        }
-    }
-
-    /**
-     * Generate crc32 for TruncateOp.
-     */
-    public static Long truncateOpCRC32(String stream, DLSN dlsn) {
-        CRC32 crc = requestCRC.get();
-        try {
-            crc.update(stream.getBytes(UTF_8));
-            crc.update(dlsn.serializeBytes());
-            return crc.getValue();
-        } finally {
-            crc.reset();
-        }
-    }
-
-    /**
-     * Generate crc32 for any op which only passes a stream name.
-     */
-    public static Long streamOpCRC32(String stream) {
-        CRC32 crc = requestCRC.get();
-        try {
-            crc.update(stream.getBytes(UTF_8));
-            return crc.getValue();
-        } finally {
-            crc.reset();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/resources/findbugsExclude.xml b/distributedlog-protocol/src/main/resources/findbugsExclude.xml
index 55e50f6..5e1cd0e 100644
--- a/distributedlog-protocol/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-protocol/src/main/resources/findbugsExclude.xml
@@ -17,10 +17,6 @@
 //-->
 <FindBugsFilter>
   <Match>
-    <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~org\.apache\.distributedlog\.thrift.*" />
-  </Match>
-  <Match>
     <!-- it is safe to store external bytes reference here. //-->
     <Class name="org.apache.distributedlog.LogRecord" />
     <Bug pattern="EI_EXPOSE_REP2" />

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/thrift/service.thrift
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/thrift/service.thrift b/distributedlog-protocol/src/main/thrift/service.thrift
deleted file mode 100644
index 45e1449..0000000
--- a/distributedlog-protocol/src/main/thrift/service.thrift
+++ /dev/null
@@ -1,203 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-namespace java org.apache.distributedlog.thrift.service
-
-/* Response stats codes */
-enum StatusCode {
-    /* 2xx: action requested by the client was received, understood, accepted and processed successfully. */
-
-    /* standard response for successful requests. */
-    SUCCESS = 200,
-
-    /* 3xx: client must take additional action to complete the request. */
-
-    /* client closed. */
-    CLIENT_CLOSED = 301,
-    /* found the stream in a different server, a redirection is required by client. */
-    FOUND = 302,
-
-    /* 4xx: client seems to have erred. */
-
-    /* request is denied for some reason */
-    REQUEST_DENIED = 403,
-    /* request record too large */
-    TOO_LARGE_RECORD = 413,
-
-    /* 5xx: server failed to fulfill an apparently valid request. */
-
-    /* Generic error message, given when no more specific message is suitable. */
-    INTERNAL_SERVER_ERROR = 500,
-    /* Not implemented */
-    NOT_IMPLEMENTED = 501,
-    /* Already Closed Exception */
-    ALREADY_CLOSED = 502,
-    /* Service is currently unavailable (because it is overloaded or down for maintenance). */
-    SERVICE_UNAVAILABLE = 503,
-    /* Locking exception */
-    LOCKING_EXCEPTION = 504,
-    /* ZooKeeper Errors */
-    ZOOKEEPER_ERROR = 505,
-    /* Metadata exception */
-    METADATA_EXCEPTION = 506,
-    /* BK Transmit Error */
-    BK_TRANSMIT_ERROR = 507,
-    /* Flush timeout */
-    FLUSH_TIMEOUT = 508,
-    /* Log empty */
-    LOG_EMPTY = 509,
-    /* Log not found */
-    LOG_NOT_FOUND = 510,
-    /* Truncated Transactions */
-    TRUNCATED_TRANSACTION = 511,
-    /* End of Stream */
-    END_OF_STREAM = 512,
-    /* Transaction Id Out of Order */
-    TRANSACTION_OUT_OF_ORDER = 513,
-    /* Write exception */
-    WRITE_EXCEPTION = 514,
-    /* Stream Unavailable */
-    STREAM_UNAVAILABLE = 515,
-    /* Write cancelled exception */
-    WRITE_CANCELLED_EXCEPTION = 516,
-    /* over-capacity/backpressure */
-    OVER_CAPACITY = 517,
-    /** stream exists but is not ready (recovering etc.).
-        the difference between NOT_READY and UNAVAILABLE is that UNAVAILABLE
-        indicates the stream is no longer owned by the proxy and we should
-        redirect. NOT_READY indicates the stream exist at the proxy but isn't
-        eady for writes. */
-    STREAM_NOT_READY = 518,
-    /* Region Unavailable */
-    REGION_UNAVAILABLE = 519,
-    /* Invalid Enveloped Entry */
-    INVALID_ENVELOPED_ENTRY = 520,
-    /* Unsupported metadata version */
-    UNSUPPORTED_METADATA_VERSION = 521,
-    /* Log Already Exists */
-    LOG_EXISTS = 522,
-    /* Checksum failed on the request */
-    CHECKSUM_FAILED = 523,
-    /* Overcapacity: too many streams */
-    TOO_MANY_STREAMS = 524,
-    /* Log Segment Not Found */
-    LOG_SEGMENT_NOT_FOUND = 525,
-    /* End of Log Segment */
-    END_OF_LOG_SEGMENT = 526,
-    /* Log Segment Is Truncated */
-    LOG_SEGMENT_IS_TRUNCATED = 527,
-
-    /* 6xx: unexpected */
-
-    UNEXPECTED = 600,
-    INTERRUPTED = 601,
-    INVALID_STREAM_NAME = 602,
-    ILLEGAL_STATE = 603,
-
-    /* 10xx: reader exceptions */
-
-    RETRYABLE_READ = 1000,
-    LOG_READ_ERROR = 1001,
-    /* Read cancelled exception */
-    READ_CANCELLED_EXCEPTION = 1002,
-}
-
-/* Response Header */
-struct ResponseHeader {
-    1: required StatusCode code;
-    2: optional string errMsg;
-    3: optional string location;
-}
-
-/* Write Response */
-struct WriteResponse {
-    1: required ResponseHeader header;
-    2: optional string dlsn;
-}
-
-/* Bulk write response */
-struct BulkWriteResponse {
-    1: required ResponseHeader header;
-    2: optional list<WriteResponse> writeResponses;
-}
-
-/* Write Context */
-struct WriteContext {
-    1: optional set<string> triedHosts;
-    2: optional i64 crc32;
-    3: optional bool isRecordSet;
-}
-
-/* HeartBeat Options */
-struct HeartbeatOptions {
-    1: optional bool sendHeartBeatToReader;
-}
-
-/* Server Status */
-enum ServerStatus {
-    /* service is writing and accepting new streams */
-    WRITE_AND_ACCEPT    = 100,
-    /* service is only writing to old streams, not accepting new streams */
-    WRITE_ONLY          = 200,
-    /* service is shutting down, will not write */
-    DOWN                = 300,
-}
-
-/* Server Info */
-struct ServerInfo {
-    1: optional map<string, string> ownerships;
-    2: optional ServerStatus serverStatus;
-}
-
-/* Client Info */
-struct ClientInfo {
-    1: optional string streamNameRegex;
-    2: optional bool getOwnerships;
-}
-
-service DistributedLogService {
-
-    /* Deprecated */
-    ServerInfo handshake();
-
-    ServerInfo handshakeWithClientInfo(ClientInfo clientInfo);
-
-    /* Deprecated */
-    WriteResponse heartbeat(string stream, WriteContext ctx);
-
-    WriteResponse heartbeatWithOptions(string stream, WriteContext ctx, HeartbeatOptions options);
-
-    /* Deprecated */
-    WriteResponse write(string stream, binary data);
-
-    WriteResponse writeWithContext(string stream, binary data, WriteContext ctx);
-
-    BulkWriteResponse writeBulkWithContext(string stream, list<binary> data, WriteContext ctx);
-
-    WriteResponse truncate(string stream, string dlsn, WriteContext ctx);
-
-    WriteResponse release(string stream, WriteContext ctx);
-
-    WriteResponse create(string stream, WriteContext ctx);
-
-    WriteResponse delete(string stream, WriteContext ctx);
-
-    WriteResponse getOwner(string stream, WriteContext ctx);
-
-    /* Admin Methods */
-    void setAcceptNewStream(bool enabled);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/pom.xml b/distributedlog-proxy-client/pom.xml
new file mode 100644
index 0000000..7392d90
--- /dev/null
+++ b/distributedlog-proxy-client/pom.xml
@@ -0,0 +1,172 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.distributedlog</groupId>
+    <artifactId>distributedlog</artifactId>
+    <version>0.5.0-incubating-SNAPSHOT</version>
+  </parent>
+  <artifactId>distributedlog-proxy-client</artifactId>
+  <name>Apache DistributedLog :: Proxy Client</name>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-proxy-protocol</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-core_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-thriftmux_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-serversets_2.11</artifactId>
+      <version>${finagle.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-jdk14</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>${guava.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <version>${commons-lang3.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>${slf4j.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <version>${mockito.version}</version>
+      <scope>test</scope>
+    </dependency> 
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-protocol</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/src/main/resources/findbugsExclude.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>${maven-compiler-plugin.version}</version>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
+          <forkMode>always</forkMode>
+          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>org.apache.distributedlog.TimedOutTestsListener</value>
+            </property>
+          </properties>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven-checkstyle-plugin.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>com.puppycrawl.tools</groupId>
+            <artifactId>checkstyle</artifactId>
+            <version>${puppycrawl.checkstyle.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.distributedlog</groupId>
+            <artifactId>distributedlog-build-tools</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <consoleOutput>true</consoleOutput>
+          <failOnViolation>true</failOnViolation>
+          <includeResources>false</includeResources>
+          <includeTestSourceDirectory>true</includeTestSourceDirectory>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>test-compile</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
new file mode 100644
index 0000000..57e2b5a
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
new file mode 100644
index 0000000..0ed93d0
--- /dev/null
+++ b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
@@ -0,0 +1,1199 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.ServiceUnavailableException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.protocol.util.ProtocolUtils;
+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 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(ProtocolUtils.exception(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, ProtocolUtils.exception(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, ProtocolUtils.exception(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, ProtocolUtils.exception(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 });
+        }
+    }
+
+}


[15/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
new file mode 100644
index 0000000..862f05a
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.util.Utils;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.List;
+import java.util.TreeSet;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Transaction;
+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;
+
+/**
+ * An implementation of the PlacementStateManager that saves data to and loads from Zookeeper to
+ * avoid necessitating an additional system for the resource placement.
+ */
+public class ZKPlacementStateManager implements PlacementStateManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(ZKPlacementStateManager.class);
+
+    private static final String SERVER_LOAD_DIR = "/.server-load";
+
+    private final String serverLoadPath;
+    private final ZooKeeperClient zkClient;
+
+    private boolean watching = false;
+
+    public ZKPlacementStateManager(URI uri, DistributedLogConfiguration conf, StatsLogger statsLogger) {
+        String zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
+        zkClient = BKNamespaceDriver.createZKClientBuilder(
+            String.format("ZKPlacementStateManager-%s", zkServers),
+            conf,
+            zkServers,
+            statsLogger.scope("placement_state_manager")).build();
+        serverLoadPath = uri.getPath() + SERVER_LOAD_DIR;
+    }
+
+    private void createServerLoadPathIfNoExists(byte[] data)
+        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
+        try {
+            Utils.zkCreateFullPathOptimistic(
+                zkClient, serverLoadPath, data, zkClient.getDefaultACL(), CreateMode.PERSISTENT);
+        } catch (KeeperException.NodeExistsException nee) {
+            logger.debug("the server load path {} is already created by others", serverLoadPath, nee);
+        }
+    }
+
+    @Override
+    public void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException {
+        logger.info("saving ownership");
+        try {
+            ZooKeeper zk = zkClient.get();
+            // use timestamp as data so watchers will see any changes
+            byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
+
+            if (zk.exists(serverLoadPath, false) == null) { //create path to rootnode if it does not yet exist
+                createServerLoadPathIfNoExists(timestamp);
+            }
+
+            Transaction tx = zk.transaction();
+            List<String> children = zk.getChildren(serverLoadPath, false);
+            HashSet<String> servers = new HashSet<String>(children);
+            tx.setData(serverLoadPath, timestamp, -1); // trigger the watcher that data has been updated
+            for (ServerLoad serverLoad : serverLoads) {
+                String server = serverToZkFormat(serverLoad.getServer());
+                String serverPath = serverPath(server);
+                if (servers.contains(server)) {
+                    servers.remove(server);
+                    tx.setData(serverPath, serverLoad.serialize(), -1);
+                } else {
+                    tx.create(serverPath, serverLoad.serialize(), zkClient.getDefaultACL(), CreateMode.PERSISTENT);
+                }
+            }
+            for (String server : servers) {
+                tx.delete(serverPath(server), -1);
+            }
+            tx.commit();
+        } catch (InterruptedException | IOException | KeeperException e) {
+            throw new StateManagerSaveException(e);
+        }
+    }
+
+    @Override
+    public TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException {
+        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
+        try {
+            ZooKeeper zk = zkClient.get();
+            List<String> children = zk.getChildren(serverLoadPath, false);
+            for (String server : children) {
+                ownerships.add(ServerLoad.deserialize(zk.getData(serverPath(server), false, new Stat())));
+            }
+            return ownerships;
+        } catch (InterruptedException | IOException | KeeperException e) {
+            throw new StateManagerLoadException(e);
+        }
+    }
+
+    @Override
+    public synchronized void watch(final PlacementCallback callback) {
+        if (watching) {
+            return; // do not double watch
+        }
+        watching = true;
+
+        try {
+            ZooKeeper zk = zkClient.get();
+            try {
+                zk.getData(serverLoadPath, new Watcher() {
+                    @Override
+                    public void process(WatchedEvent watchedEvent) {
+                        try {
+                            callback.callback(loadOwnership());
+                        } catch (StateManagerLoadException e) {
+                            logger.error("Watch of Ownership failed", e);
+                        } finally {
+                            watching = false;
+                            watch(callback);
+                        }
+                    }
+                }, new Stat());
+            } catch (KeeperException.NoNodeException nee) {
+                byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
+                createServerLoadPathIfNoExists(timestamp);
+                watching = false;
+                watch(callback);
+            }
+        } catch (ZooKeeperClient.ZooKeeperConnectionException | InterruptedException | KeeperException e) {
+            logger.error("Watch of Ownership failed", e);
+            watching = false;
+            watch(callback);
+        }
+    }
+
+    public String serverPath(String server) {
+        return String.format("%s/%s", serverLoadPath, server);
+    }
+
+    protected String serverToZkFormat(String server) {
+        return server.replaceAll("/", "--");
+    }
+
+    protected String zkFormatToServer(String zkFormattedServer) {
+        return zkFormattedServer.replaceAll("--", "/");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/package-info.java
new file mode 100644
index 0000000..ea79251
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/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.
+ */
+/**
+ * Placement Policy to place streams across proxy services.
+ */
+package org.apache.distributedlog.service.placement;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
new file mode 100644
index 0000000..83ac668
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.exceptions.ChecksumFailedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import com.twitter.util.Return;
+import com.twitter.util.Try;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * Abstract Stream Operation.
+ */
+public abstract class AbstractStreamOp<Response> implements StreamOp {
+
+    private static final Logger logger = LoggerFactory.getLogger(AbstractStreamOp.class);
+
+    protected final String stream;
+    protected final OpStatsLogger opStatsLogger;
+    private final Promise<Response> result = new Promise<Response>();
+    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
+    protected final Long checksum;
+    protected final Feature checksumDisabledFeature;
+
+    public AbstractStreamOp(String stream,
+                            OpStatsLogger statsLogger,
+                            Long checksum,
+                            Feature checksumDisabledFeature) {
+        this.stream = stream;
+        this.opStatsLogger = statsLogger;
+        // start here in case the operation is failed before executing.
+        stopwatch.reset().start();
+        this.checksum = checksum;
+        this.checksumDisabledFeature = checksumDisabledFeature;
+    }
+
+    @Override
+    public String streamName() {
+        return stream;
+    }
+
+    @Override
+    public Stopwatch stopwatch() {
+        return stopwatch;
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
+            Long serverChecksum = computeChecksum();
+            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
+                throw new ChecksumFailedException();
+            }
+        }
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return null;
+    }
+
+    @Override
+    public Future<Void> execute(AsyncLogWriter writer, Sequencer sequencer, Object txnLock) {
+        stopwatch.reset().start();
+        return executeOp(writer, sequencer, txnLock)
+                .addEventListener(new FutureEventListener<Response>() {
+            @Override
+            public void onSuccess(Response response) {
+                opStatsLogger.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                setResponse(response);
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+            }
+        }).voided();
+    }
+
+    /**
+     * Fail with current <i>owner</i> and its reason <i>t</i>.
+     *
+     * @param cause
+     *          failure reason
+     */
+    @Override
+    public void fail(Throwable cause) {
+        if (cause instanceof OwnershipAcquireFailedException) {
+            // Ownership exception is a control exception, not an error, so we don't stat
+            // it with the other errors.
+            OwnershipAcquireFailedException oafe = (OwnershipAcquireFailedException) cause;
+            fail(ResponseUtils.ownerToHeader(oafe.getCurrentOwner()));
+        } else {
+            opStatsLogger.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+            fail(ResponseUtils.exceptionToHeader(cause));
+        }
+    }
+
+    protected void setResponse(Response response) {
+      Return<Response> responseTry = new Return(response);
+      boolean isEmpty = result.updateIfEmpty(responseTry);
+      if (!isEmpty) {
+        Option<Try<Response>> resultTry = result.poll();
+        logger.error("Result set multiple times. Value='{}', New='{}'", resultTry, responseTry);
+      }
+    }
+
+    /**
+     * Return the full response, header and body.
+     *
+     * @return A future containing the response or the exception
+     *      encountered by the op if it failed.
+     */
+    public Future<Response> result() {
+        return result;
+    }
+
+    /**
+     * Execute the operation and return its corresponding response.
+     *
+     * @param writer
+     *          writer to execute the operation.
+     * @param sequencer
+     *          sequencer used for generating transaction id for stream operations
+     * @param txnLock
+     *          transaction lock to guarantee ordering of transaction id
+     * @return future representing the operation.
+     */
+    protected abstract Future<Response> executeOp(AsyncLogWriter writer,
+                                                  Sequencer sequencer,
+                                                  Object txnLock);
+
+    // fail the result with the given response header
+    protected abstract void fail(ResponseHeader header);
+
+    public static OpStatsLogger requestStat(StatsLogger statsLogger, String opName) {
+        return requestLogger(statsLogger).getOpStatsLogger(opName);
+    }
+
+    public static StatsLogger requestLogger(StatsLogger statsLogger) {
+        return statsLogger.scope("request");
+    }
+
+    public static StatsLogger requestScope(StatsLogger statsLogger, String scope) {
+        return requestLogger(statsLogger).scope(scope);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java
new file mode 100644
index 0000000..8befffc
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.protocol.util.ProtocolUtils;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Abstract Write Operation.
+ */
+public abstract class AbstractWriteOp extends AbstractStreamOp<WriteResponse> {
+
+    protected AbstractWriteOp(String stream,
+                              OpStatsLogger statsLogger,
+                              Long checksum,
+                              Feature checksumDisabledFeature) {
+        super(stream, statsLogger, checksum, checksumDisabledFeature);
+    }
+
+    @Override
+    protected void fail(ResponseHeader header) {
+        setResponse(ResponseUtils.write(header));
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return ProtocolUtils.streamOpCRC32(stream);
+    }
+
+    @Override
+    public Future<ResponseHeader> responseHeader() {
+        return result().map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
+            @Override
+            public ResponseHeader apply(WriteResponse response) {
+                return response.getHeader();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
new file mode 100644
index 0000000..6c98468
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
@@ -0,0 +1,253 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.RequestDeniedException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.ConstFuture;
+import com.twitter.util.Future;
+import com.twitter.util.Future$;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Try;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Bulk Write Operation.
+ */
+public class BulkWriteOp extends AbstractStreamOp<BulkWriteResponse> implements WriteOpWithPayload {
+    private final List<ByteBuffer> buffers;
+    private final long payloadSize;
+
+    // Stats
+    private final Counter deniedBulkWriteCounter;
+    private final Counter successRecordCounter;
+    private final Counter failureRecordCounter;
+    private final Counter redirectRecordCounter;
+    private final OpStatsLogger latencyStat;
+    private final Counter bytes;
+    private final Counter bulkWriteBytes;
+
+    private final AccessControlManager accessControlManager;
+
+    // We need to pass these through to preserve ownership change behavior in
+    // client/server. Only include failures which are guaranteed to have failed
+    // all subsequent writes.
+    private boolean isDefiniteFailure(Try<DLSN> result) {
+        boolean def = false;
+        try {
+            result.get();
+        } catch (Exception ex) {
+            if (ex instanceof OwnershipAcquireFailedException
+                || ex instanceof AlreadyClosedException
+                || ex instanceof LockingException) {
+                def = true;
+            }
+        }
+        return def;
+    }
+
+    public BulkWriteOp(String stream,
+                       List<ByteBuffer> buffers,
+                       StatsLogger statsLogger,
+                       StatsLogger perStreamStatsLogger,
+                       StreamPartitionConverter streamPartitionConverter,
+                       Long checksum,
+                       Feature checksumDisabledFeature,
+                       AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "bulkWrite"), checksum, checksumDisabledFeature);
+        this.buffers = buffers;
+        long total = 0;
+        // We do this here because the bytebuffers are mutable.
+        for (ByteBuffer bb : buffers) {
+          total += bb.remaining();
+        }
+        this.payloadSize = total;
+
+        final Partition partition = streamPartitionConverter.convert(stream);
+        // Write record stats
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedBulkWriteCounter = streamOpStats.requestDeniedCounter("bulkWrite");
+        this.successRecordCounter = streamOpStats.recordsCounter("success");
+        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
+        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
+        this.bulkWriteBytes = streamOpStats.scopedRequestCounter("bulkWrite", "bytes");
+        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "bulkWrite");
+        this.bytes = streamOpStats.streamRequestCounter(partition, "bulkWrite", "bytes");
+
+        this.accessControlManager = accessControlManager;
+
+        final long size = getPayloadSize();
+        result().addEventListener(new FutureEventListener<BulkWriteResponse>() {
+            @Override
+            public void onSuccess(BulkWriteResponse response) {
+                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                    bytes.add(size);
+                    bulkWriteBytes.add(size);
+                } else {
+                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                }
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowWrite(stream)) {
+            deniedBulkWriteCounter.inc();
+            throw new RequestDeniedException(stream, "bulkWrite");
+        }
+        super.preExecute();
+    }
+
+    @Override
+    public long getPayloadSize() {
+      return payloadSize;
+    }
+
+    @Override
+    protected Future<BulkWriteResponse> executeOp(AsyncLogWriter writer,
+                                                  Sequencer sequencer,
+                                                  Object txnLock) {
+        // Need to convert input buffers to LogRecords.
+        List<LogRecord> records;
+        Future<List<Future<DLSN>>> futureList;
+        synchronized (txnLock) {
+            records = asRecordList(buffers, sequencer);
+            futureList = writer.writeBulk(records);
+        }
+
+        // Collect into a list of tries to make it easier to extract exception or DLSN.
+        Future<List<Try<DLSN>>> writes = asTryList(futureList);
+
+        Future<BulkWriteResponse> response = writes.flatMap(
+            new AbstractFunction1<List<Try<DLSN>>, Future<BulkWriteResponse>>() {
+                @Override
+                public Future<BulkWriteResponse> apply(List<Try<DLSN>> results) {
+
+                    // Considered a success at batch level even if no individual writes succeeed.
+                    // The reason is that its impossible to make an appropriate decision re retries without
+                    // individual buffer failure reasons.
+                    List<WriteResponse> writeResponses = new ArrayList<WriteResponse>(results.size());
+                    BulkWriteResponse bulkWriteResponse =
+                        ResponseUtils.bulkWriteSuccess().setWriteResponses(writeResponses);
+
+                    // Promote the first result to an op-level failure if we're sure all other writes have
+                    // failed.
+                    if (results.size() > 0) {
+                        Try<DLSN> firstResult = results.get(0);
+                        if (isDefiniteFailure(firstResult)) {
+                            return new ConstFuture(firstResult);
+                        }
+                    }
+
+                    // Translate all futures to write responses.
+                    Iterator<Try<DLSN>> iterator = results.iterator();
+                    while (iterator.hasNext()) {
+                        Try<DLSN> completedFuture = iterator.next();
+                        try {
+                            DLSN dlsn = completedFuture.get();
+                            WriteResponse writeResponse = ResponseUtils.writeSuccess().setDlsn(dlsn.serialize());
+                            writeResponses.add(writeResponse);
+                            successRecordCounter.inc();
+                        } catch (Exception ioe) {
+                            WriteResponse writeResponse = ResponseUtils.write(ResponseUtils.exceptionToHeader(ioe));
+                            writeResponses.add(writeResponse);
+                            if (StatusCode.FOUND == writeResponse.getHeader().getCode()) {
+                                redirectRecordCounter.inc();
+                            } else {
+                                failureRecordCounter.inc();
+                            }
+                        }
+                    }
+
+                    return Future.value(bulkWriteResponse);
+                }
+            }
+        );
+
+        return response;
+    }
+
+    private List<LogRecord> asRecordList(List<ByteBuffer> buffers, Sequencer sequencer) {
+        List<LogRecord> records = new ArrayList<LogRecord>(buffers.size());
+        for (ByteBuffer buffer : buffers) {
+            byte[] payload = new byte[buffer.remaining()];
+            buffer.get(payload);
+            records.add(new LogRecord(sequencer.nextId(), payload));
+        }
+        return records;
+    }
+
+    private Future<List<Try<DLSN>>> asTryList(Future<List<Future<DLSN>>> futureList) {
+        return futureList.flatMap(new AbstractFunction1<List<Future<DLSN>>, Future<List<Try<DLSN>>>>() {
+            @Override
+            public Future<List<Try<DLSN>>> apply(List<Future<DLSN>> results) {
+                return Future$.MODULE$.collectToTry(results);
+            }
+        });
+    }
+
+    @Override
+    protected void fail(ResponseHeader header) {
+        if (StatusCode.FOUND == header.getCode()) {
+            redirectRecordCounter.add(buffers.size());
+        } else {
+            failureRecordCounter.add(buffers.size());
+        }
+        setResponse(ResponseUtils.bulkWrite(header));
+    }
+
+    @Override
+    public Future<ResponseHeader> responseHeader() {
+        return result().map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
+            @Override
+            public ResponseHeader apply(BulkWriteResponse response) {
+                return response.getHeader();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
new file mode 100644
index 0000000..3ecb46f
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.RequestDeniedException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to delete a log stream.
+ */
+public class DeleteOp extends AbstractWriteOp {
+    private final StreamManager streamManager;
+    private final Counter deniedDeleteCounter;
+    private final AccessControlManager accessControlManager;
+
+    public DeleteOp(String stream,
+                    StatsLogger statsLogger,
+                    StatsLogger perStreamStatsLogger,
+                    StreamManager streamManager,
+                    Long checksum,
+                    Feature checksumEnabledFeature,
+                    AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "delete"), checksum, checksumEnabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedDeleteCounter = streamOpStats.requestDeniedCounter("delete");
+        this.accessControlManager = accessControlManager;
+        this.streamManager = streamManager;
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        Future<Void> result = streamManager.deleteAndRemoveAsync(streamName());
+        return result.map(new AbstractFunction1<Void, WriteResponse>() {
+            @Override
+            public WriteResponse apply(Void value) {
+                return ResponseUtils.writeSuccess();
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowTruncate(stream)) {
+            deniedDeleteCounter.inc();
+            throw new RequestDeniedException(stream, "delete");
+        }
+        super.preExecute();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
new file mode 100644
index 0000000..0ffa619
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.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.service.stream;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.RequestDeniedException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Heartbeat Operation.
+ */
+public class HeartbeatOp extends AbstractWriteOp {
+
+    static final byte[] HEARTBEAT_DATA = "heartbeat".getBytes(UTF_8);
+
+    private final AccessControlManager accessControlManager;
+    private final Counter deniedHeartbeatCounter;
+    private final byte dlsnVersion;
+
+    private boolean writeControlRecord = false;
+
+    public HeartbeatOp(String stream,
+                       StatsLogger statsLogger,
+                       StatsLogger perStreamStatsLogger,
+                       byte dlsnVersion,
+                       Long checksum,
+                       Feature checksumDisabledFeature,
+                       AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "heartbeat"), checksum, checksumDisabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedHeartbeatCounter = streamOpStats.requestDeniedCounter("heartbeat");
+        this.dlsnVersion = dlsnVersion;
+        this.accessControlManager = accessControlManager;
+    }
+
+    public HeartbeatOp setWriteControlRecord(boolean writeControlRecord) {
+        this.writeControlRecord = writeControlRecord;
+        return this;
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        // write a control record if heartbeat is the first request of the recovered log segment.
+        if (writeControlRecord) {
+            long txnId;
+            Future<DLSN> writeResult;
+            synchronized (txnLock) {
+                txnId = sequencer.nextId();
+                LogRecord hbRecord = new LogRecord(txnId, HEARTBEAT_DATA);
+                hbRecord.setControl();
+                writeResult = writer.write(hbRecord);
+            }
+            return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
+                @Override
+                public WriteResponse apply(DLSN value) {
+                    return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
+                }
+            });
+        } else {
+            return Future.value(ResponseUtils.writeSuccess());
+        }
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowAcquire(stream)) {
+            deniedHeartbeatCounter.inc();
+            throw new RequestDeniedException(stream, "heartbeat");
+        }
+        super.preExecute();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
new file mode 100644
index 0000000..6ec8642
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.RequestDeniedException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to release ownership of a log stream.
+ */
+public class ReleaseOp extends AbstractWriteOp {
+    private final StreamManager streamManager;
+    private final Counter deniedReleaseCounter;
+    private final AccessControlManager accessControlManager;
+
+    public ReleaseOp(String stream,
+                     StatsLogger statsLogger,
+                     StatsLogger perStreamStatsLogger,
+                     StreamManager streamManager,
+                     Long checksum,
+                     Feature checksumDisabledFeature,
+                     AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "release"), checksum, checksumDisabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedReleaseCounter = streamOpStats.requestDeniedCounter("release");
+        this.accessControlManager = accessControlManager;
+        this.streamManager = streamManager;
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        Future<Void> result = streamManager.closeAndRemoveAsync(streamName());
+        return result.map(new AbstractFunction1<Void, WriteResponse>() {
+            @Override
+            public WriteResponse apply(Void value) {
+                return ResponseUtils.writeSuccess();
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowRelease(stream)) {
+            deniedReleaseCounter.inc();
+            throw new RequestDeniedException(stream, "release");
+        }
+        super.preExecute();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/Stream.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/Stream.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/Stream.java
new file mode 100644
index 0000000..3517a63
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/Stream.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.service.streamset.Partition;
+import com.twitter.util.Future;
+import java.io.IOException;
+
+/**
+ * Stream is the per stream request handler in the DL service layer.
+ *
+ * <p>The collection of Streams in the proxy are managed by StreamManager.
+ */
+public interface Stream {
+
+    /**
+     * Get the stream configuration for this stream.
+     *
+     * @return stream configuration
+     */
+    DynamicDistributedLogConfiguration getStreamConfiguration();
+
+    /**
+     * Get the stream's last recorded current owner (may be out of date). Used
+     * as a hint for the client.
+     * @return last known owner for the stream
+     */
+    String getOwner();
+
+    /**
+     * Get the stream name.
+     * @return stream name
+     */
+    String getStreamName();
+
+    /**
+     * Get the represented partition name.
+     *
+     * @return represented partition name.
+     */
+    Partition getPartition();
+
+    /**
+     * Expensive initialization code run after stream has been allocated in
+     * StreamManager.
+     *
+     * @throws IOException when encountered exception on initialization
+     */
+    void initialize() throws IOException;
+
+    /**
+     * Another initialize method (actually Thread.start). Should probably be
+     * moved to initialize().
+     */
+    void start();
+
+    /**
+     * Asynchronous close method.
+     * @param reason for closing
+     * @return future satisfied once close complete
+     */
+    Future<Void> requestClose(String reason);
+
+    /**
+     * Delete the stream from DL backend.
+     *
+     * @throws IOException when encountered exception on deleting the stream.
+     */
+    void delete() throws IOException;
+
+    /**
+     * Execute the stream operation against this stream.
+     *
+     * @param op operation to execute
+     */
+    void submit(StreamOp op);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java
new file mode 100644
index 0000000..845ef21
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
+/**
+ * Factory to create a stream with provided stream configuration {@code streamConf}.
+ */
+public interface StreamFactory {
+
+    /**
+     * Create a stream object.
+     *
+     * @param name stream name
+     * @param streamConf stream configuration
+     * @param streamManager manager of streams
+     * @return stream object
+     */
+    Stream create(String name,
+                  DynamicDistributedLogConfiguration streamConf,
+                  StreamManager streamManager);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
new file mode 100644
index 0000000..2b90d55
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.service.FatalErrorHandler;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.util.OrderedScheduler;
+import com.twitter.util.Timer;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.jboss.netty.util.HashedWheelTimer;
+
+/**
+ * The implementation of {@link StreamFactory}.
+ */
+public class StreamFactoryImpl implements StreamFactory {
+    private final String clientId;
+    private final StreamOpStats streamOpStats;
+    private final ServerConfiguration serverConfig;
+    private final DistributedLogConfiguration dlConfig;
+    private final FeatureProvider featureProvider;
+    private final StreamConfigProvider streamConfigProvider;
+    private final StreamPartitionConverter streamPartitionConverter;
+    private final DistributedLogNamespace dlNamespace;
+    private final OrderedScheduler scheduler;
+    private final FatalErrorHandler fatalErrorHandler;
+    private final HashedWheelTimer requestTimer;
+    private final Timer futureTimer;
+
+    public StreamFactoryImpl(String clientId,
+        StreamOpStats streamOpStats,
+        ServerConfiguration serverConfig,
+        DistributedLogConfiguration dlConfig,
+        FeatureProvider featureProvider,
+        StreamConfigProvider streamConfigProvider,
+        StreamPartitionConverter streamPartitionConverter,
+        DistributedLogNamespace dlNamespace,
+        OrderedScheduler scheduler,
+        FatalErrorHandler fatalErrorHandler,
+        HashedWheelTimer requestTimer) {
+
+        this.clientId = clientId;
+        this.streamOpStats = streamOpStats;
+        this.serverConfig = serverConfig;
+        this.dlConfig = dlConfig;
+        this.featureProvider = featureProvider;
+        this.streamConfigProvider = streamConfigProvider;
+        this.streamPartitionConverter = streamPartitionConverter;
+        this.dlNamespace = dlNamespace;
+        this.scheduler = scheduler;
+        this.fatalErrorHandler = fatalErrorHandler;
+        this.requestTimer = requestTimer;
+        this.futureTimer = new com.twitter.finagle.util.HashedWheelTimer(requestTimer);
+    }
+
+    @Override
+    public Stream create(String name,
+                         DynamicDistributedLogConfiguration streamConf,
+                         StreamManager streamManager) {
+        return new StreamImpl(name,
+            streamPartitionConverter.convert(name),
+            clientId,
+            streamManager,
+            streamOpStats,
+            serverConfig,
+            dlConfig,
+            streamConf,
+            featureProvider,
+            streamConfigProvider,
+            dlNamespace,
+            scheduler,
+            fatalErrorHandler,
+            requestTimer,
+            futureTimer);
+    }
+}


[13/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
new file mode 100644
index 0000000..c3c5d81
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.admin;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.exceptions.ChecksumFailedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.protocol.util.ProtocolUtils;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import com.twitter.util.Future;
+import com.twitter.util.FutureTransformer;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+
+/**
+ * Stream admin op.
+ */
+public abstract class StreamAdminOp implements AdminOp<WriteResponse> {
+
+    protected final String stream;
+    protected final StreamManager streamManager;
+    protected final OpStatsLogger opStatsLogger;
+    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
+    protected final Long checksum;
+    protected final Feature checksumDisabledFeature;
+
+    protected StreamAdminOp(String stream,
+                            StreamManager streamManager,
+                            OpStatsLogger statsLogger,
+                            Long checksum,
+                            Feature checksumDisabledFeature) {
+        this.stream = stream;
+        this.streamManager = streamManager;
+        this.opStatsLogger = statsLogger;
+        // start here in case the operation is failed before executing.
+        stopwatch.reset().start();
+        this.checksum = checksum;
+        this.checksumDisabledFeature = checksumDisabledFeature;
+    }
+
+    protected Long computeChecksum() {
+        return ProtocolUtils.streamOpCRC32(stream);
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
+            Long serverChecksum = computeChecksum();
+            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
+                throw new ChecksumFailedException();
+            }
+        }
+    }
+
+    /**
+     * Execute the operation.
+     *
+     * @return execute operation
+     */
+    protected abstract Future<WriteResponse> executeOp();
+
+    @Override
+    public Future<WriteResponse> execute() {
+        return executeOp().transformedBy(new FutureTransformer<WriteResponse, WriteResponse>() {
+
+            @Override
+            public WriteResponse map(WriteResponse response) {
+                opStatsLogger.registerSuccessfulEvent(
+                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                return response;
+            }
+
+            @Override
+            public WriteResponse handle(Throwable cause) {
+                opStatsLogger.registerFailedEvent(
+                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                return ResponseUtils.write(ResponseUtils.exceptionToHeader(cause));
+            }
+
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java
new file mode 100644
index 0000000..5b583e1
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/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.
+ */
+/**
+ * Stream Related Admin Operations.
+ */
+package org.apache.distributedlog.service.stream.admin;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
new file mode 100644
index 0000000..5db2037
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.limiter;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import java.io.Closeable;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.commons.configuration.event.ConfigurationEvent;
+import org.apache.commons.configuration.event.ConfigurationListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Dynamically rebuild a rate limiter when the supplied dynamic config changes.
+ *
+ * <p>Subclasses implement build() to build the limiter. DynamicRequestLimiter must be closed to deregister
+ * the config listener.
+ */
+public abstract class DynamicRequestLimiter<Req> implements RequestLimiter<Req>, Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(DynamicRequestLimiter.class);
+
+    private final ConfigurationListener listener;
+    private final Feature rateLimitDisabledFeature;
+    volatile RequestLimiter<Req> limiter;
+    final DynamicDistributedLogConfiguration dynConf;
+
+    public DynamicRequestLimiter(DynamicDistributedLogConfiguration dynConf,
+                                 StatsLogger statsLogger,
+                                 Feature rateLimitDisabledFeature) {
+        final StatsLogger limiterStatsLogger = statsLogger.scope("dynamic");
+        this.dynConf = dynConf;
+        this.rateLimitDisabledFeature = rateLimitDisabledFeature;
+        this.listener = new ConfigurationListener() {
+            @Override
+            public void configurationChanged(ConfigurationEvent event) {
+                // Note that this method may be called several times if several config options
+                // are changed. The effect is harmless except that we create and discard more
+                // objects than we need to.
+                LOG.debug("Config changed callback invoked with event {} {} {} {}", new Object[] {
+                        event.getPropertyName(), event.getPropertyValue(), event.getType(),
+                        event.isBeforeUpdate()});
+                if (!event.isBeforeUpdate()) {
+                    limiterStatsLogger.getCounter("config_changed").inc();
+                    LOG.debug("Rebuilding limiter");
+                    limiter = build();
+                }
+            }
+        };
+        LOG.debug("Registering config changed callback");
+        dynConf.addConfigurationListener(listener);
+    }
+
+    public void initialize() {
+        this.limiter = build();
+    }
+
+    @Override
+    public void apply(Req request) throws OverCapacityException {
+        if (rateLimitDisabledFeature.isAvailable()) {
+            return;
+        }
+        limiter.apply(request);
+    }
+
+    @Override
+    public void close() {
+        boolean success = dynConf.removeConfigurationListener(listener);
+        LOG.debug("Deregistering config changed callback success={}", success);
+    }
+
+   /**
+    * Build the underlying limiter. Called when DynamicRequestLimiter detects config has changed.
+    * This may be called multiple times so the method should be cheap.
+    */
+    protected abstract RequestLimiter<Req> build();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
new file mode 100644
index 0000000..fc30599
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.limiter;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
+import org.apache.distributedlog.limiter.GuavaRateLimiter;
+import org.apache.distributedlog.limiter.RateLimiter;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.distributedlog.service.stream.WriteOpWithPayload;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Request limiter builder.
+ */
+public class RequestLimiterBuilder {
+    private OverlimitFunction<StreamOp> overlimitFunction = NOP_OVERLIMIT_FUNCTION;
+    private RateLimiter limiter;
+    private CostFunction<StreamOp> costFunction;
+    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+
+    /**
+     * Function to calculate the `RPS` (Request per second) cost of a given stream operation.
+     */
+    public static final CostFunction<StreamOp> RPS_COST_FUNCTION = new CostFunction<StreamOp>() {
+        @Override
+        public int apply(StreamOp op) {
+            if (op instanceof WriteOpWithPayload) {
+                return 1;
+            } else {
+                return 0;
+            }
+        }
+    };
+
+    /**
+     * Function to calculate the `BPS` (Bytes per second) cost of a given stream operation.
+     */
+    public static final CostFunction<StreamOp> BPS_COST_FUNCTION = new CostFunction<StreamOp>() {
+        @Override
+        public int apply(StreamOp op) {
+            if (op instanceof WriteOpWithPayload) {
+                WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
+                return (int) Math.min(writeOp.getPayloadSize(), Integer.MAX_VALUE);
+            } else {
+                return 0;
+            }
+        }
+    };
+
+    /**
+     * Function to check if a stream operation will cause {@link OverCapacityException}.
+     */
+    public static final OverlimitFunction<StreamOp> NOP_OVERLIMIT_FUNCTION = new OverlimitFunction<StreamOp>() {
+        @Override
+        public void apply(StreamOp op) throws OverCapacityException {
+            return;
+        }
+    };
+
+    public RequestLimiterBuilder limit(int limit) {
+        this.limiter = GuavaRateLimiter.of(limit);
+        return this;
+    }
+
+    public RequestLimiterBuilder overlimit(OverlimitFunction<StreamOp> overlimitFunction) {
+        this.overlimitFunction = overlimitFunction;
+        return this;
+    }
+
+    public RequestLimiterBuilder cost(CostFunction<StreamOp> costFunction) {
+        this.costFunction = costFunction;
+        return this;
+    }
+
+    public RequestLimiterBuilder statsLogger(StatsLogger statsLogger) {
+        this.statsLogger = statsLogger;
+        return this;
+    }
+
+    public static RequestLimiterBuilder newRpsLimiterBuilder() {
+        return new RequestLimiterBuilder().cost(RPS_COST_FUNCTION);
+    }
+
+    public static RequestLimiterBuilder newBpsLimiterBuilder() {
+        return new RequestLimiterBuilder().cost(BPS_COST_FUNCTION);
+    }
+
+    public RequestLimiter<StreamOp> build() {
+        checkNotNull(limiter);
+        checkNotNull(overlimitFunction);
+        checkNotNull(costFunction);
+        return new ComposableRequestLimiter(limiter, overlimitFunction, costFunction, statsLogger);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
new file mode 100644
index 0000000..de805aa
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.limiter;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.ChainedRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import org.apache.distributedlog.rate.MovingAverageRate;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Request limiter for the service instance (global request limiter).
+ */
+public class ServiceRequestLimiter extends DynamicRequestLimiter<StreamOp> {
+    private final StatsLogger limiterStatLogger;
+    private final MovingAverageRate serviceRps;
+    private final MovingAverageRate serviceBps;
+    private final StreamManager streamManager;
+
+    public ServiceRequestLimiter(DynamicDistributedLogConfiguration dynConf,
+                                 StatsLogger statsLogger,
+                                 MovingAverageRate serviceRps,
+                                 MovingAverageRate serviceBps,
+                                 StreamManager streamManager,
+                                 Feature disabledFeature) {
+        super(dynConf, statsLogger, disabledFeature);
+        this.limiterStatLogger = statsLogger;
+        this.streamManager = streamManager;
+        this.serviceRps = serviceRps;
+        this.serviceBps = serviceBps;
+        this.limiter = build();
+    }
+
+    @Override
+    public RequestLimiter<StreamOp> build() {
+        int rpsStreamAcquireLimit = dynConf.getRpsStreamAcquireServiceLimit();
+        int rpsSoftServiceLimit = dynConf.getRpsSoftServiceLimit();
+        int rpsHardServiceLimit = dynConf.getRpsHardServiceLimit();
+        int bpsStreamAcquireLimit = dynConf.getBpsStreamAcquireServiceLimit();
+        int bpsSoftServiceLimit = dynConf.getBpsSoftServiceLimit();
+        int bpsHardServiceLimit = dynConf.getBpsHardServiceLimit();
+
+        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
+            .limit(rpsHardServiceLimit)
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp request) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for the service instance");
+                }
+            });
+
+        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
+            .limit(rpsSoftServiceLimit);
+
+        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
+            .limit(bpsHardServiceLimit)
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp request) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for the service instance");
+                }
+            });
+
+        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
+            .limit(bpsSoftServiceLimit);
+
+        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
+        builder.addLimiter(new StreamAcquireLimiter(
+            streamManager, serviceRps, rpsStreamAcquireLimit, limiterStatLogger.scope("rps_acquire")));
+        builder.addLimiter(new StreamAcquireLimiter(
+            streamManager, serviceBps, bpsStreamAcquireLimit, limiterStatLogger.scope("bps_acquire")));
+        builder.addLimiter(bpsHardLimiterBuilder.build());
+        builder.addLimiter(bpsSoftLimiterBuilder.build());
+        builder.addLimiter(rpsHardLimiterBuilder.build());
+        builder.addLimiter(rpsSoftLimiterBuilder.build());
+        builder.statsLogger(limiterStatLogger);
+        return builder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
new file mode 100644
index 0000000..7675d6f
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.limiter;
+
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.exceptions.TooManyStreamsException;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import org.apache.distributedlog.rate.MovingAverageRate;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * A special limiter on limiting acquiring new streams.
+ */
+public class StreamAcquireLimiter implements RequestLimiter<StreamOp> {
+    private final StreamManager streamManager;
+    private final MovingAverageRate serviceRps;
+    private final double serviceRpsLimit;
+    private final Counter overlimitCounter;
+
+    public StreamAcquireLimiter(StreamManager streamManager,
+                                MovingAverageRate serviceRps,
+                                double serviceRpsLimit,
+                                StatsLogger statsLogger) {
+        this.streamManager = streamManager;
+        this.serviceRps = serviceRps;
+        this.serviceRpsLimit = serviceRpsLimit;
+        this.overlimitCounter = statsLogger.getCounter("overlimit");
+    }
+
+    @Override
+    public void apply(StreamOp op) throws OverCapacityException {
+        String streamName = op.streamName();
+        if (serviceRpsLimit > -1 && serviceRps.get() > serviceRpsLimit && !streamManager.isAcquired(streamName)) {
+            overlimitCounter.inc();
+            throw new TooManyStreamsException("Request rate is too high to accept new stream " + streamName + ".");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java
new file mode 100644
index 0000000..42b4e1e
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.limiter;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.ChainedRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * A dynamic request limiter on limiting stream operations.
+ */
+public class StreamRequestLimiter extends DynamicRequestLimiter<StreamOp> {
+    private final DynamicDistributedLogConfiguration dynConf;
+    private final StatsLogger limiterStatLogger;
+    private final String streamName;
+
+    public StreamRequestLimiter(String streamName,
+                                DynamicDistributedLogConfiguration dynConf,
+                                StatsLogger statsLogger,
+                                Feature disabledFeature) {
+        super(dynConf, statsLogger, disabledFeature);
+        this.limiterStatLogger = statsLogger;
+        this.dynConf = dynConf;
+        this.streamName = streamName;
+        this.limiter = build();
+    }
+
+    @Override
+    public RequestLimiter<StreamOp> build() {
+
+        // RPS hard, soft limits
+        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
+            .limit(dynConf.getRpsHardWriteLimit())
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp op) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for stream " + streamName);
+                }
+            });
+        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
+            .limit(dynConf.getRpsSoftWriteLimit());
+
+        // BPS hard, soft limits
+        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
+            .limit(dynConf.getBpsHardWriteLimit())
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp op) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for stream " + streamName);
+                }
+            });
+        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
+            .limit(dynConf.getBpsSoftWriteLimit());
+
+        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
+        builder.addLimiter(rpsSoftLimiterBuilder.build());
+        builder.addLimiter(rpsHardLimiterBuilder.build());
+        builder.addLimiter(bpsSoftLimiterBuilder.build());
+        builder.addLimiter(bpsHardLimiterBuilder.build());
+        builder.statsLogger(limiterStatLogger);
+        return builder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java
new file mode 100644
index 0000000..c666b08
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/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.
+ */
+/**
+ * Request Rate Limiting.
+ */
+package org.apache.distributedlog.service.stream.limiter;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/package-info.java
new file mode 100644
index 0000000..7429a85
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/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 Related Operations.
+ */
+package org.apache.distributedlog.service.stream;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
new file mode 100644
index 0000000..72668c2
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.streamset;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * A stream-to-partition converter that caches the mapping between stream and partitions.
+ */
+public abstract class CacheableStreamPartitionConverter implements StreamPartitionConverter {
+
+    private final ConcurrentMap<String, Partition> partitions;
+
+    protected CacheableStreamPartitionConverter() {
+        this.partitions = new ConcurrentHashMap<String, Partition>();
+    }
+
+    @Override
+    public Partition convert(String streamName) {
+        Partition p = partitions.get(streamName);
+        if (null != p) {
+            return p;
+        }
+        // not found
+        Partition newPartition = newPartition(streamName);
+        Partition oldPartition = partitions.putIfAbsent(streamName, newPartition);
+        if (null == oldPartition) {
+            return newPartition;
+        } else {
+            return oldPartition;
+        }
+    }
+
+    /**
+     * Create the partition from <code>streamName</code>.
+     *
+     * @param streamName
+     *          stream name
+     * @return partition id of the stream
+     */
+    protected abstract Partition newPartition(String streamName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
new file mode 100644
index 0000000..30b2896
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.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.service.streamset;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Stream Partition Converter that converts the stream name into a stream-to-partition mapping by delimiter.
+ */
+public class DelimiterStreamPartitionConverter extends CacheableStreamPartitionConverter {
+
+    private final String delimiter;
+
+    public DelimiterStreamPartitionConverter() {
+        this("_");
+    }
+
+    public DelimiterStreamPartitionConverter(String delimiter) {
+        this.delimiter = delimiter;
+    }
+
+    @Override
+    protected Partition newPartition(String streamName) {
+        String[] parts = StringUtils.split(streamName, delimiter);
+        if (null != parts && parts.length == 2) {
+            try {
+                int partition = Integer.parseInt(parts[1]);
+                return new Partition(parts[0], partition);
+            } catch (NumberFormatException nfe) {
+                // ignore the exception
+            }
+        }
+        return new Partition(streamName, 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
new file mode 100644
index 0000000..5be172f
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.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.service.streamset;
+
+/**
+ * Map stream name to partition of the same name.
+ */
+public class IdentityStreamPartitionConverter extends CacheableStreamPartitionConverter {
+    @Override
+    protected Partition newPartition(String streamName) {
+        return new Partition(streamName, 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/Partition.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
new file mode 100644
index 0000000..aa69276
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.streamset;
+
+import com.google.common.base.Objects;
+
+/**
+ * `Partition` defines the relationship between a `virtual` stream and a
+ * physical DL stream.
+ *
+ * <p>A `virtual` stream could be partitioned into multiple partitions
+ * and each partition is effectively a DL stream.
+ */
+public class Partition {
+
+    // Name of its parent stream.
+    private final String stream;
+
+    // Unique id of the partition within the stream.
+    // It can be just simply an index id.
+    public final int id;
+
+    public Partition(String stream, int id) {
+        this.stream = stream;
+        this.id = id;
+    }
+
+    /**
+     * Get the `virtual` stream name.
+     *
+     * @return the stream name.
+     */
+    public String getStream() {
+        return stream;
+    }
+
+    /**
+     * Get the partition id of this partition.
+     *
+     * @return partition id
+     */
+    public int getId() {
+        return id;
+    }
+
+    /**
+     * Get the 6 digit 0 padded id of this partition as a String.
+     * @return partition id
+     */
+    public String getPaddedId() {
+        return String.format("%06d", getId());
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof Partition)) {
+            return false;
+        }
+        Partition partition = (Partition) o;
+
+        return id == partition.id && Objects.equal(stream, partition.stream);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = stream.hashCode();
+        result = 31 * result + id;
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("Partition(")
+          .append(stream)
+          .append(", ")
+          .append(id)
+          .append(")");
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java
new file mode 100644
index 0000000..bfcc5db
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.streamset;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A mapping between a logical stream and a set of physical partitions.
+ */
+public class PartitionMap {
+
+    private final Map<String, Set<Partition>> partitionMap;
+
+    public PartitionMap() {
+        partitionMap = new HashMap<String, Set<Partition>>();
+    }
+
+    public synchronized boolean addPartition(Partition partition, int maxPartitions) {
+        if (maxPartitions <= 0) {
+            return true;
+        }
+        Set<Partition> partitions = partitionMap.get(partition.getStream());
+        if (null == partitions) {
+            partitions = new HashSet<Partition>();
+            partitions.add(partition);
+            partitionMap.put(partition.getStream(), partitions);
+            return true;
+        }
+        if (partitions.contains(partition) || partitions.size() < maxPartitions) {
+            partitions.add(partition);
+            return true;
+        }
+        return false;
+    }
+
+    public synchronized boolean removePartition(Partition partition) {
+        Set<Partition> partitions = partitionMap.get(partition.getStream());
+        return null != partitions && partitions.remove(partition);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java
new file mode 100644
index 0000000..3ea1337
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.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.service.streamset;
+
+/**
+ * Map stream name to a partition.
+ *
+ * @see Partition
+ */
+public interface StreamPartitionConverter {
+
+    /**
+     * Convert the stream name to partition.
+     *
+     * @param streamName
+     *          stream name
+     * @return partition
+     */
+    Partition convert(String streamName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/package-info.java
new file mode 100644
index 0000000..d185e88
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/streamset/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.
+ */
+/**
+ * StreamSet - A logical set of streams.
+ */
+package org.apache.distributedlog.service.streamset;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
new file mode 100644
index 0000000..3934eb5
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
@@ -0,0 +1,350 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.tools;
+
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.service.ClientUtils;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.tools.Tool;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tools to interact with proxies.
+ */
+public class ProxyTool extends Tool {
+
+    private static final Logger logger = LoggerFactory.getLogger(ProxyTool.class);
+
+    /**
+     * Abstract Cluster level command.
+     */
+    protected abstract static class ClusterCommand extends OptsCommand {
+
+        protected Options options = new Options();
+        protected URI uri;
+        protected final List<String> streams = new ArrayList<String>();
+
+        protected ClusterCommand(String name, String description) {
+            super(name, description);
+            options.addOption("u", "uri", true, "DistributedLog URI");
+            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 '0-9', list '1,2,3' and name '143'");
+        }
+
+        @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;
+            }
+
+            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
+            logger.info("Created serverset for {}", uri);
+            try {
+                DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                        .name("proxy_tool")
+                        .clientId(ClientId$.MODULE$.apply("proxy_tool"))
+                        .maxRedirects(2)
+                        .serverSet(serverSet.getServerSet())
+                        .clientBuilder(ClientBuilder.get()
+                            .connectionTimeout(Duration.fromSeconds(2))
+                            .tcpConnectTimeout(Duration.fromSeconds(2))
+                            .requestTimeout(Duration.fromSeconds(10))
+                            .hostConnectionLimit(1)
+                            .hostConnectionCoresize(1)
+                            .keepAlive(true)
+                            .failFast(false))
+                        .build();
+                try {
+                    return runCmd(client);
+                } finally {
+                    client.close();
+                }
+            } finally {
+                serverSet.close();
+            }
+        }
+
+        protected abstract int runCmd(DistributedLogClient client) 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.");
+            }
+            this.uri = URI.create(cmdline.getOptionValue("u"));
+
+            // get stream names
+            String streamPrefix = cmdline.hasOption("r") ? cmdline.getOptionValue("r") : "";
+            String streamExpression = null;
+            if (cmdline.hasOption("e")) {
+                streamExpression = cmdline.getOptionValue("e");
+            }
+            if (null == streamPrefix || null == streamExpression) {
+                throw new ParseException("Please specify stream prefix & expression.");
+            }
+            // 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) {
+                        streams.add(streamPrefix + part);
+                    }
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid stream suffix list : " + streamExpression);
+                }
+            } else {
+                streams.add(streamPrefix + streamExpression);
+            }
+        }
+    }
+
+    /**
+     * Command to release ownership of a log stream.
+     */
+    static class ReleaseCommand extends ClusterCommand {
+
+        double rate = 100f;
+
+        ReleaseCommand() {
+            super("release", "Release Stream Ownerships");
+            options.addOption("t", "rate", true, "Rate to release streams");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("t")) {
+                rate = Double.parseDouble(cmdline.getOptionValue("t", "100"));
+            }
+        }
+
+        @Override
+        protected int runCmd(DistributedLogClient client) throws Exception {
+            RateLimiter rateLimiter = RateLimiter.create(rate);
+            for (String stream : streams) {
+                rateLimiter.acquire();
+                try {
+                    Await.result(client.release(stream));
+                    System.out.println("Release ownership of stream " + stream);
+                } catch (Exception e) {
+                    System.err.println("Failed to release ownership of stream " + stream);
+                    throw e;
+                }
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "release [options]";
+        }
+    }
+
+    /**
+     * Command to truncate a log stream.
+     */
+    static class TruncateCommand extends ClusterCommand {
+
+        DLSN dlsn = DLSN.InitialDLSN;
+
+        TruncateCommand() {
+            super("truncate", "Truncate streams until given dlsn.");
+            options.addOption("d", "dlsn", true, "DLSN to truncate until");
+        }
+
+        @Override
+        protected int runCmd(DistributedLogClient client) throws Exception {
+            System.out.println("Truncating streams : " + streams);
+            for (String stream : streams) {
+                boolean success = Await.result(client.truncate(stream, dlsn));
+                System.out.println("Truncate " + stream + " to " + dlsn + " : " + success);
+            }
+            return 0;
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("d")) {
+                throw new ParseException("No DLSN provided");
+            }
+            String[] dlsnStrs = cmdline.getOptionValue("d").split(",");
+            if (dlsnStrs.length != 3) {
+                throw new ParseException("Invalid DLSN : " + cmdline.getOptionValue("d"));
+            }
+            dlsn = new DLSN(Long.parseLong(dlsnStrs[0]), Long.parseLong(dlsnStrs[1]), Long.parseLong(dlsnStrs[2]));
+        }
+
+        @Override
+        protected String getUsage() {
+            return "truncate [options]";
+        }
+    }
+
+    /**
+     * Abstract command to operate on a single proxy server.
+     */
+    protected abstract static class ProxyCommand extends OptsCommand {
+
+        protected Options options = new Options();
+        protected InetSocketAddress address;
+
+        protected ProxyCommand(String name, String description) {
+            super(name, description);
+            options.addOption("H", "host", true, "Single Proxy Address");
+        }
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (!cmdline.hasOption("H")) {
+                throw new ParseException("No proxy address provided");
+            }
+            address = DLSocketAddress.parseSocketAddress(cmdline.getOptionValue("H"));
+        }
+
+        @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;
+            }
+
+            DistributedLogClientBuilder clientBuilder = DistributedLogClientBuilder.newBuilder()
+                    .name("proxy_tool")
+                    .clientId(ClientId$.MODULE$.apply("proxy_tool"))
+                    .maxRedirects(2)
+                    .host(address)
+                    .clientBuilder(ClientBuilder.get()
+                            .connectionTimeout(Duration.fromSeconds(2))
+                            .tcpConnectTimeout(Duration.fromSeconds(2))
+                            .requestTimeout(Duration.fromSeconds(10))
+                            .hostConnectionLimit(1)
+                            .hostConnectionCoresize(1)
+                            .keepAlive(true)
+                            .failFast(false));
+            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
+                    ClientUtils.buildClient(clientBuilder);
+            try {
+                return runCmd(clientPair);
+            } finally {
+                clientPair.getLeft().close();
+            }
+        }
+
+        protected abstract int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client) throws Exception;
+    }
+
+    /**
+     * Command to enable/disable accepting new streams.
+     */
+    static class AcceptNewStreamCommand extends ProxyCommand {
+
+        boolean enabled = false;
+
+        AcceptNewStreamCommand() {
+            super("accept-new-stream", "Enable/Disable accepting new streams for one proxy");
+            options.addOption("e", "enabled", true, "Enable/Disable accepting new streams");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("e")) {
+                throw new ParseException("No action 'enable/disable' provided");
+            }
+            enabled = Boolean.parseBoolean(cmdline.getOptionValue("e"));
+        }
+
+        @Override
+        protected int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client)
+                throws Exception {
+            Await.result(client.getRight().setAcceptNewStream(enabled));
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "accept-new-stream [options]";
+        }
+    }
+
+    public ProxyTool() {
+        super();
+        addCommand(new ReleaseCommand());
+        addCommand(new TruncateCommand());
+        addCommand(new AcceptNewStreamCommand());
+    }
+
+    @Override
+    protected String getName() {
+        return "proxy_tool";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/package-info.java
new file mode 100644
index 0000000..92d0a7d
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/tools/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.
+ */
+/**
+ * Service related tools.
+ */
+package org.apache.distributedlog.service.tools;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java
new file mode 100644
index 0000000..9ee93b4
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.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.utils;
+
+import java.io.IOException;
+import java.net.InetAddress;
+
+/**
+ * Utils that used by servers.
+ */
+public class ServerUtils {
+
+  /**
+   * Retrieve the ledger allocator pool name.
+   *
+   * @param serverRegionId region id that that server is running
+   * @param shardId shard id of the server
+   * @param useHostname whether to use hostname as the ledger allocator pool name
+   * @return ledger allocator pool name
+   * @throws IOException
+   */
+    public static String getLedgerAllocatorPoolName(int serverRegionId,
+                                                    int shardId,
+                                                    boolean useHostname)
+        throws IOException {
+        if (useHostname) {
+            return String.format("allocator_%04d_%s", serverRegionId,
+                InetAddress.getLocalHost().getHostAddress());
+        } else {
+            return String.format("allocator_%04d_%010d", serverRegionId, shardId);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/utils/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/utils/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/utils/package-info.java
new file mode 100644
index 0000000..99cf736
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/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.
+ */
+/**
+ * Utilities used by proxy servers.
+ */
+package org.apache.distributedlog.service.utils;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/resources/config/server_decider.conf
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/resources/config/server_decider.conf b/distributedlog-proxy-server/src/main/resources/config/server_decider.conf
new file mode 100644
index 0000000..d2fddf5
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/resources/config/server_decider.conf
@@ -0,0 +1,31 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+region_stop_accept_new_stream=0
+disable_durability_enforcement=0
+disable_write_limit=0
+bkc.repp_disable_durability_enforcement=0
+bkc.disable_ensemble_change=0
+dl.disable_logsegment_rolling=0
+dl.disable_write_limit=0
+bkc.atla.disallow_bookie_placement=0
+bkc.atlb.disallow_bookie_placement=0
+bkc.smf1.disallow_bookie_placement=0
+service_rate_limit_disabled=0
+service_checksum_disabled=0
+service_global_limiter_disabled=0

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/resources/config/server_decider.yml
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/resources/config/server_decider.yml b/distributedlog-proxy-server/src/main/resources/config/server_decider.yml
new file mode 100644
index 0000000..7df24bb
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/resources/config/server_decider.yml
@@ -0,0 +1,44 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+region_stop_accept_new_stream:
+  default_availability: 0
+disable_durability_enforcement:
+  default_availability: 0
+disable_write_limit:
+  default_availability: 0
+bkc.repp_disable_durability_enforcement:
+  default_availability: 0
+bkc.disable_ensemble_change:
+  default_availability: 0
+dl.disable_logsegment_rolling:
+  default_availability: 0
+dl.disable_write_limit:
+  default_availability: 0
+bkc.atla.disallow_bookie_placement:
+  default_availability: 0
+bkc.atlb.disallow_bookie_placement:
+  default_availability: 0
+bkc.smf1.disallow_bookie_placement:
+  default_availability: 0
+service_rate_limit_disabled:
+  default_availability: 0
+service_checksum_disabled:
+  default_availability: 0
+service_global_limiter_disabled:
+  default_availability: 0

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/resources/findbugsExclude.xml b/distributedlog-proxy-server/src/main/resources/findbugsExclude.xml
new file mode 100644
index 0000000..e101a4d
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/resources/findbugsExclude.xml
@@ -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.
+//-->
+<FindBugsFilter>
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
+  </Match>
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.service\.placement\.thrift.*" />
+  </Match>
+  <Match>
+    <!-- it is safe to cast exception here. //-->
+    <Class name="org.apache.distributedlog.service.DistributedLogServiceImpl$Stream$2" />
+    <Method name="onFailure" />
+    <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
+  <Match>
+    <!-- it is safe to cast exception here. //-->
+    <Class name="org.apache.distributedlog.service.stream.BulkWriteOp" />
+    <Method name="isDefiniteFailure" />
+    <Bug pattern="BC_IMPOSSIBLE_INSTANCEOF" />
+  </Match>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/thrift/metadata.thrift
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/thrift/metadata.thrift b/distributedlog-proxy-server/src/main/thrift/metadata.thrift
new file mode 100644
index 0000000..9cb3c72
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/thrift/metadata.thrift
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+namespace java org.apache.distributedlog.service.placement.thrift
+
+struct StreamLoad {
+    1: optional string stream
+    2: optional i32 load
+}
+
+struct ServerLoad {
+    1: optional string server
+    2: optional i64 load
+    3: optional list<StreamLoad> streams
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java
new file mode 100644
index 0000000..a9ddae5
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.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 com.google.common.collect.Sets;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * A local routing service that used for testing.
+ */
+public class LocalRoutingService implements RoutingService {
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder to build a local routing service for testing.
+     */
+    public static class Builder implements RoutingService.Builder {
+
+        private Builder() {}
+
+        @Override
+        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+            return this;
+        }
+
+        @Override
+        public LocalRoutingService build() {
+            return new LocalRoutingService();
+        }
+    }
+
+    private final Map<String, LinkedHashSet<SocketAddress>> localAddresses =
+            new HashMap<String, LinkedHashSet<SocketAddress>>();
+    private final CopyOnWriteArrayList<RoutingListener> listeners =
+            new CopyOnWriteArrayList<RoutingListener>();
+
+    boolean allowRetrySameHost = true;
+
+    @Override
+    public void startService() {
+        // nop
+    }
+
+    @Override
+    public void stopService() {
+        // nop
+    }
+
+    @Override
+    public synchronized Set<SocketAddress> getHosts() {
+        Set<SocketAddress> hosts = Sets.newHashSet();
+        for (LinkedHashSet<SocketAddress> addresses : localAddresses.values()) {
+            hosts.addAll(addresses);
+        }
+        return hosts;
+    }
+
+    @Override
+    public RoutingService registerListener(RoutingListener listener) {
+        listeners.add(listener);
+        return this;
+    }
+
+    @Override
+    public RoutingService unregisterListener(RoutingListener listener) {
+        listeners.remove(listener);
+        return this;
+    }
+
+    public LocalRoutingService setAllowRetrySameHost(boolean enabled) {
+        allowRetrySameHost = enabled;
+        return this;
+    }
+
+    public LocalRoutingService addHost(String stream, SocketAddress address) {
+        boolean notify = false;
+        synchronized (this) {
+            LinkedHashSet<SocketAddress> addresses = localAddresses.get(stream);
+            if (null == addresses) {
+                addresses = new LinkedHashSet<SocketAddress>();
+                localAddresses.put(stream, addresses);
+            }
+            if (addresses.add(address)) {
+                notify = true;
+            }
+        }
+        if (notify) {
+            for (RoutingListener listener : listeners) {
+                listener.onServerJoin(address);
+            }
+        }
+        return this;
+    }
+
+    @Override
+    public synchronized SocketAddress getHost(String key, RoutingContext rContext)
+            throws NoBrokersAvailableException {
+        LinkedHashSet<SocketAddress> addresses = localAddresses.get(key);
+
+        SocketAddress candidate = null;
+        if (null != addresses) {
+            for (SocketAddress host : addresses) {
+                if (rContext.isTriedHost(host) && !allowRetrySameHost) {
+                    continue;
+                } else {
+                    candidate = host;
+                    break;
+                }
+            }
+        }
+        if (null != candidate) {
+            return candidate;
+        }
+        throw new NoBrokersAvailableException("No host available");
+    }
+
+    @Override
+    public void removeHost(SocketAddress address, Throwable reason) {
+        // nop
+    }
+}


[10/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
new file mode 100644
index 0000000..431bfa4
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
@@ -0,0 +1,301 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream.limiter;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.ChainedRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
+import org.apache.distributedlog.limiter.GuavaRateLimiter;
+import org.apache.distributedlog.limiter.RateLimiter;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServiceRequestLimiter}.
+ */
+public class TestServiceRequestLimiter {
+
+    /**
+     * Mock Request.
+     */
+    class MockRequest {
+        int size;
+        MockRequest() {
+            this(1);
+        }
+        MockRequest(int size) {
+            this.size = size;
+        }
+        int getSize() {
+            return size;
+        }
+    }
+
+    /**
+     * Mock request limiter.
+     */
+    class MockRequestLimiter implements RequestLimiter<MockRequest> {
+        public void apply(MockRequest request) {
+        }
+    }
+
+    /**
+     * Counter based limiter.
+     */
+    static class CounterLimiter implements RateLimiter {
+        final int limit;
+        int count;
+
+        public CounterLimiter(int limit) {
+            this.limit = limit;
+            this.count = 0;
+        }
+
+        @Override
+        public boolean acquire(int permits) {
+            if (++count > limit) {
+                return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * Mock hard request limiter.
+     */
+    class MockHardRequestLimiter implements RequestLimiter<MockRequest> {
+
+        RequestLimiter<MockRequest> limiter;
+        int limitHitCount;
+
+        MockHardRequestLimiter(int limit) {
+            this(GuavaRateLimiter.of(limit));
+        }
+
+        MockHardRequestLimiter(RateLimiter limiter) {
+            this.limiter = new ComposableRequestLimiter<MockRequest>(
+                limiter,
+                new OverlimitFunction<MockRequest>() {
+                    public void apply(MockRequest request) throws OverCapacityException {
+                        limitHitCount++;
+                        throw new OverCapacityException("Limit exceeded");
+                    }
+                },
+                new CostFunction<MockRequest>() {
+                    public int apply(MockRequest request) {
+                        return request.getSize();
+                    }
+                },
+                NullStatsLogger.INSTANCE);
+        }
+
+        @Override
+        public void apply(MockRequest op) throws OverCapacityException {
+            limiter.apply(op);
+        }
+
+        public int getLimitHitCount() {
+            return limitHitCount;
+        }
+    }
+
+    /**
+     * Mock soft request limiter.
+     */
+    class MockSoftRequestLimiter implements RequestLimiter<MockRequest> {
+
+        RequestLimiter<MockRequest> limiter;
+        int limitHitCount;
+
+        MockSoftRequestLimiter(int limit) {
+            this(GuavaRateLimiter.of(limit));
+        }
+
+        MockSoftRequestLimiter(RateLimiter limiter) {
+            this.limiter = new ComposableRequestLimiter<MockRequest>(
+                limiter,
+                new OverlimitFunction<MockRequest>() {
+                    public void apply(MockRequest request) throws OverCapacityException {
+                        limitHitCount++;
+                    }
+                },
+                new CostFunction<MockRequest>() {
+                    public int apply(MockRequest request) {
+                        return request.getSize();
+                    }
+                },
+                NullStatsLogger.INSTANCE);
+        }
+
+        @Override
+        public void apply(MockRequest op) throws OverCapacityException {
+            limiter.apply(op);
+        }
+
+        public int getLimitHitCount() {
+            return limitHitCount;
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testDynamicLimiter() throws Exception {
+        final AtomicInteger id = new AtomicInteger(0);
+        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
+        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
+                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
+            @Override
+            public RequestLimiter<MockRequest> build() {
+                id.getAndIncrement();
+                return new MockRequestLimiter();
+            }
+        };
+        limiter.initialize();
+        assertEquals(1, id.get());
+        dynConf.setProperty("test1", 1);
+        assertEquals(2, id.get());
+        dynConf.setProperty("test2", 2);
+        assertEquals(3, id.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testDynamicLimiterWithDisabledFeature() throws Exception {
+        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
+        final MockSoftRequestLimiter rateLimiter = new MockSoftRequestLimiter(0);
+        final SettableFeature disabledFeature = new SettableFeature("", 0);
+        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
+                dynConf, NullStatsLogger.INSTANCE, disabledFeature) {
+            @Override
+            public RequestLimiter<MockRequest> build() {
+                return rateLimiter;
+            }
+        };
+        limiter.initialize();
+        assertEquals(0, rateLimiter.getLimitHitCount());
+
+        // Not disabled, rate limiter was invoked
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        assertEquals(1, rateLimiter.getLimitHitCount());
+
+        // Disabled, rate limiter not invoked
+        disabledFeature.set(1);
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        assertEquals(1, rateLimiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testDynamicLimiterWithException() throws Exception {
+        final AtomicInteger id = new AtomicInteger(0);
+        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
+        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
+                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
+            @Override
+            public RequestLimiter<MockRequest> build() {
+                if (id.incrementAndGet() >= 2) {
+                    throw new RuntimeException("exception in dynamic limiter build()");
+                }
+                return new MockRequestLimiter();
+            }
+        };
+        limiter.initialize();
+        assertEquals(1, id.get());
+        try {
+            dynConf.setProperty("test1", 1);
+            fail("should have thrown on config failure");
+        } catch (RuntimeException ex) {
+        }
+        assertEquals(2, id.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceRequestLimiter() throws Exception {
+        MockHardRequestLimiter limiter = new MockHardRequestLimiter(new CounterLimiter(1));
+        limiter.apply(new MockRequest());
+        try {
+            limiter.apply(new MockRequest());
+        } catch (OverCapacityException ex) {
+        }
+        assertEquals(1, limiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceRequestLimiterWithDefaultRate() throws Exception {
+        MockHardRequestLimiter limiter = new MockHardRequestLimiter(-1);
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        assertEquals(0, limiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceRequestLimiterWithZeroRate() throws Exception {
+        MockHardRequestLimiter limiter = new MockHardRequestLimiter(0);
+        try {
+            limiter.apply(new MockRequest(1));
+            fail("should have failed with overcap");
+        } catch (OverCapacityException ex) {
+        }
+        assertEquals(1, limiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testChainedServiceRequestLimiter() throws Exception {
+        MockSoftRequestLimiter softLimiter = new MockSoftRequestLimiter(new CounterLimiter(1));
+        MockHardRequestLimiter hardLimiter = new MockHardRequestLimiter(new CounterLimiter(3));
+
+        RequestLimiter<MockRequest> limiter =
+                new ChainedRequestLimiter.Builder<MockRequest>()
+                .addLimiter(softLimiter)
+                .addLimiter(hardLimiter)
+                .build();
+
+        assertEquals(0, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        limiter.apply(new MockRequest());
+        assertEquals(0, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        limiter.apply(new MockRequest());
+        assertEquals(1, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        limiter.apply(new MockRequest());
+        assertEquals(2, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        try {
+            limiter.apply(new MockRequest());
+        } catch (OverCapacityException ex) {
+        }
+        assertEquals(3, softLimiter.getLimitHitCount());
+        assertEquals(1, hardLimiter.getLimitHitCount());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
new file mode 100644
index 0000000..15a0753
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.streamset;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+/**
+ * Test Cases for {@link DelimiterStreamPartitionConverter}.
+ */
+public class TestDelimiterStreamPartitionConverter {
+
+    @Test(timeout = 20000)
+    public void testNormalStream() throws Exception {
+        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
+        assertEquals(new Partition("distributedlog-smoketest", 1),
+                converter.convert("distributedlog-smoketest_1"));
+        assertEquals(new Partition("distributedlog-smoketest-", 1),
+                converter.convert("distributedlog-smoketest-_1"));
+        assertEquals(new Partition("distributedlog-smoketest", 1),
+                converter.convert("distributedlog-smoketest_000001"));
+    }
+
+    private void assertIdentify(String streamName, StreamPartitionConverter converter) {
+        assertEquals(new Partition(streamName, 0), converter.convert(streamName));
+    }
+
+    @Test(timeout = 20000)
+    public void testUnknownStream() throws Exception {
+        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
+        assertIdentify("test1", converter);
+        assertIdentify("test1-000001", converter);
+        assertIdentify("test1_test1_000001", converter);
+        assertIdentify("test1_test1", converter);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
new file mode 100644
index 0000000..1a5d8d3
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.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.service.streamset;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+/**
+ * Test Case for {@link IdentityStreamPartitionConverter}.
+ */
+public class TestIdentityStreamPartitionConverter {
+
+    @Test(timeout = 20000)
+    public void testIdentityConverter() {
+        String streamName = "test-identity-converter";
+
+        IdentityStreamPartitionConverter converter =
+                new IdentityStreamPartitionConverter();
+
+        Partition p0 = converter.convert(streamName);
+        assertEquals(new Partition(streamName, 0), p0);
+
+        Partition p1 = converter.convert(streamName);
+        assertTrue(p0 == p1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java
new file mode 100644
index 0000000..b6e5ff3
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.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.service.streamset;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+/**
+ * Test {@link PartitionMap}.
+ */
+public class TestPartitionMap {
+
+    @Test(timeout = 20000)
+    public void testAddPartitionNegativeMax() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 20; i++) {
+            assertTrue(map.addPartition(new Partition("test", i), -1));
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testAddPartitionMultipleTimes() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 20; i++) {
+            assertTrue(map.addPartition(new Partition("test", 0), 3));
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testAddPartition() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 3; i++) {
+            assertTrue(map.addPartition(new Partition("test", i), 3));
+        }
+        for (int i = 3; i < 20; i++) {
+            assertFalse(map.addPartition(new Partition("test", i), 3));
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testRemovePartition() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 3; i++) {
+            assertTrue(map.addPartition(new Partition("test", i), 3));
+        }
+        assertFalse(map.addPartition(new Partition("test", 3), 3));
+        assertFalse(map.removePartition(new Partition("test", 3)));
+        assertTrue(map.removePartition(new Partition("test", 0)));
+        assertTrue(map.addPartition(new Partition("test", 3), 3));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java
new file mode 100644
index 0000000..2853df1
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.utils;
+
+import static org.junit.Assert.assertEquals;
+
+import java.net.InetAddress;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServerUtils}.
+ */
+public class TestServerUtils {
+
+    @Test(timeout = 6000)
+    public void testGetLedgerAllocatorPoolName() throws Exception {
+        int region = 123;
+        int shard = 999;
+        String hostname = InetAddress.getLocalHost().getHostAddress();
+        assertEquals("allocator_0123_0000000999",
+            ServerUtils.getLedgerAllocatorPoolName(region, shard, false));
+        assertEquals("allocator_0123_" + hostname,
+            ServerUtils.getLedgerAllocatorPoolName(region, shard, true));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/resources/log4j.properties b/distributedlog-proxy-server/src/test/resources/log4j.properties
new file mode 100644
index 0000000..3e51059
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/resources/log4j.properties
@@ -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.
+# */
+
+#
+# DisributedLog Logging Configuration
+#
+
+# Example with rolling log file
+log4j.rootLogger=INFO, CONSOLE
+
+#disable zookeeper logging
+log4j.logger.org.apache.zookeeper=OFF
+#Set the bookkeeper level to warning
+log4j.logger.org.apache.bookkeeper=INFO
+
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.Threshold=INFO
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+# Add ROLLINGFILE to rootLogger to get log file output
+#    Log DEBUG level and above messages to a log file
+#log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
+#log4j.appender.ROLLINGFILE.Threshold=INFO
+#log4j.appender.ROLLINGFILE.File=distributedlog.log
+#log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+#log4j.appender.ROLLINGFILE.DatePattern='.'yyyy-MM-dd-HH-mm
+#log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+log4j.appender.R=org.apache.log4j.RollingFileAppender
+log4j.appender.R.Threshold=TRACE
+log4j.appender.R.File=target/error.log
+log4j.appender.R.MaxFileSize=200MB
+log4j.appender.R.MaxBackupIndex=7
+log4j.appender.R.layout=org.apache.log4j.PatternLayout
+log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/bin/bundle
----------------------------------------------------------------------
diff --git a/distributedlog-service/bin/bundle b/distributedlog-service/bin/bundle
deleted file mode 100755
index 1b1904e..0000000
--- a/distributedlog-service/bin/bundle
+++ /dev/null
@@ -1,28 +0,0 @@
-#!/bin/bash
-#
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-set -e
-
-BASEDIR=$(dirname "$0")
-DLOG_ROOT="${BASEDIR}/../.."
-
-cd "${DLOG_ROOT}"
-
-bash scripts/bundle service

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/bin/common.sh
----------------------------------------------------------------------
diff --git a/distributedlog-service/bin/common.sh b/distributedlog-service/bin/common.sh
deleted file mode 100755
index 2b13157..0000000
--- a/distributedlog-service/bin/common.sh
+++ /dev/null
@@ -1,124 +0,0 @@
-#!/usr/bin/env bash
-#
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-if [ $(uname) == "Linux" ]; then
-  # check if net.ipv6.bindv6only is set to 1
-  bindv6only=$(/sbin/sysctl -n net.ipv6.bindv6only 2> /dev/null)
-  if [ -n "${bindv6only}" ] && [ "${bindv6only}" -eq "1" ]; then
-    echo "Error: \"net.ipv6.bindv6only\" is set to 1 - Java networking could be broken"
-    echo "For more info (the following page also applies to DistributedLog): http://wiki.apache.org/hadoop/HadoopIPv6"
-    exit 1
-  fi
-fi
-
-# See the following page for extensive details on setting
-# up the JVM to accept JMX remote management:
-# http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
-# by default we allow local JMX connections
-if [ -z "${JMXLOCALONLY}" ]; then
-  JMXLOCALONLY=false
-fi
-
-if [ -z "${JMXDISABLE}" ]; then
-  echo "JMX enabled by default" >&2
-  # for some reason these two options are necessary on jdk6 on Ubuntu
-  # accord to the docs they are not necessary, but otw jconsole cannot
-  # do a local attach
-  JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=${JMXLOCALONLY}"
-else
-  echo "JMX disabled by user request" >&2
-fi
-
-echo "DLOG_HOME => ${DLOG_HOME}"
-DEFAULT_LOG_CONF="${DLOG_HOME}/conf/log4j.properties"
-
-[ -f "${DLOG_HOME}/conf/dlogenv.sh" ] && source "${DLOG_HOME}/conf/dlogenv.sh"
-
-# exclude tests jar
-RELEASE_JAR=$(ls ${DLOG_HOME}/distributedlog-*.jar 2> /dev/null | grep -v 'tests\|javadoc\|sources' | tail -1)
-if [ $? == 0 ]; then
-  DLOG_JAR="${RELEASE_JAR}"
-fi
-
-# exclude tests jar
-BUILT_JAR=$(ls ${DLOG_HOME}/target/distributedlog-*.jar 2> /dev/null | grep -v 'tests\|javadoc\|sources' | tail -1)
-
-if [ -e "${BUILD_JAR}" ] && [ -e "${DLOG_JAR}" ]; then
-  echo "\nCouldn't find dlog jar.";
-  echo "Make sure you've run 'mvn package'\n";
-  exit 1;
-elif [ -e "${BUILT_JAR}" ]; then
-  DLOG_JAR="${BUILT_JAR}"
-fi
-
-add_maven_deps_to_classpath() {
-  MVN="mvn"
-  if [ -n "${MAVEN_HOME}" ]; then
-    MVN="${MAVEN_HOME}/bin/mvn"
-  fi
-
-  # Need to generate classpath from maven pom. This is costly so generate it
-  # and cache it. Save the file into our target dir so a mvn clean will get
-  # clean it up and force us create a new one.
-  f="${PWD}/${DLOG_HOME}/target/cached_classpath.txt"
-  if [ ! -f "${f}" ]; then
-    "${MVN}" -f "${DLOG_HOME}/pom.xml" dependency:build-classpath -Dmdep.outputFile="${f}" &> /dev/null
-  fi
-  DLOG_CLASSPATH="${CLASSPATH}":$(cat "${f}")
-}
-
-if [ -d "${DLOG_HOME}/lib" ]; then
-  for i in ${DLOG_HOME}/lib/*.jar; do
-    DLOG_CLASSPATH="${DLOG_CLASSPATH}:${i}"
-  done
-else
-  add_maven_deps_to_classpath
-fi
-
-# if no args specified, exit
-if [ $# = 0 ]; then
-  exit 1
-fi
-
-if [ -z "${DLOG_LOG_CONF}" ]; then
-  DLOG_LOG_CONF="${DEFAULT_LOG_CONF}"
-fi
-
-DLOG_CLASSPATH="${DLOG_JAR}:${DLOG_CLASSPATH}:${DLOG_EXTRA_CLASSPATH}"
-if [ -n "${DLOG_LOG_CONF}" ]; then
-  DLOG_CLASSPATH="$(dirname ${DLOG_LOG_CONF}):${DLOG_CLASSPATH}"
-  OPTS="${OPTS} -Dlog4j.configuration=$(basename ${DLOG_LOG_CONF})"
-fi
-OPTS="-cp ${DLOG_CLASSPATH} ${OPTS} ${DLOG_EXTRA_OPTS}"
-
-OPTS="${OPTS} ${DLOG_EXTRA_OPTS}"
-
-# Disable ipv6 as it can cause issues
-OPTS="${OPTS} -Djava.net.preferIPv4Stack=true"
-
-# log directory & file
-DLOG_ROOT_LOGGER=${DLOG_ROOT_LOGGER:-"INFO,R"}
-DLOG_LOG_DIR=${DLOG_LOG_DIR:-"$DLOG_HOME/logs"}
-DLOG_LOG_FILE=${DLOG_LOG_FILE:-"dlog.log"}
-
-#Configure log configuration system properties
-OPTS="$OPTS -Ddlog.root.logger=${DLOG_ROOT_LOGGER}"
-OPTS="$OPTS -Ddlog.log.dir=${DLOG_LOG_DIR}"
-OPTS="$OPTS -Ddlog.log.file=${DLOG_LOG_FILE}"

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/bin/dlog
----------------------------------------------------------------------
diff --git a/distributedlog-service/bin/dlog b/distributedlog-service/bin/dlog
deleted file mode 100755
index 99cad40..0000000
--- a/distributedlog-service/bin/dlog
+++ /dev/null
@@ -1,88 +0,0 @@
-#!/usr/bin/env bash
-#
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-set -e
-
-BASEDIR=$(dirname "$0")
-DLOG_HOME="${BASEDIR}/.."
-
-usage() {
-  cat <<EOF
-Usage: runner <command>
-where command is one of:
-  local               Run distributedlog sandbox
-  example             Run distributedlog example
-  tool                Run distributedlog tool
-  proxy_tool          Run distributedlog proxy tool to interact with proxies
-  balancer            Run distributedlog balancer
-  admin               Run distributedlog admin tool
-  zkshell             Run zookeeper shell
-  bkshell             Run bookkeeper shell
-  help                This help message
-
-or command is the full name of a class with a defined main() method.
-
-Environment variables:
-  DLOG_LOG_CONF        Log4j configuration file (default $DEFAULT_LOG_CONF)
-  DLOG_EXTRA_OPTS      Extra options to be passed to the jvm
-  DLOG_EXTRA_CLASSPATH Add extra paths to the dlog classpath
-
-These variable can also be set in conf/dlogenv.sh
-EOF
-}
-
-source "${DLOG_HOME}"/bin/common.sh
-
-# get arguments
-COMMAND=$1
-shift
-
-case "${COMMAND}" in
-  local)
-    exec java $OPTS $JMX_ARGS org.apache.distributedlog.LocalDLMEmulator $@
-    ;;
-  tool)
-    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.tools.DistributedLogTool $@
-    ;;
-  proxy_tool)
-    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.service.tools.ProxyTool $@
-    ;;
-  balancer)
-    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.service.balancer.BalancerTool $@
-    ;;
-  admin)
-    exec java $OPTS org.apache.distributedlog.tools.Tool org.apache.distributedlog.admin.DistributedLogAdmin $@
-    ;;
-  zkshell)
-    exec java $OPTS org.apache.zookeeper.ZooKeeperMain -server $@
-    ;;
-  bkshell)
-    ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}"
-    exec java $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF $@
-    ;;
-  help)
-    usage
-    ;;
-  *)
-    exec java $OPTS $COMMAND $@
-    ;;
-esac
-
-

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/bin/dlog-daemon.sh
----------------------------------------------------------------------
diff --git a/distributedlog-service/bin/dlog-daemon.sh b/distributedlog-service/bin/dlog-daemon.sh
deleted file mode 100755
index fdd87df..0000000
--- a/distributedlog-service/bin/dlog-daemon.sh
+++ /dev/null
@@ -1,200 +0,0 @@
-#!/usr/bin/env bash
-#
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-usage() {
-  cat <<EOF
-Usage: dlog-daemon.sh (start|stop) <service> <args...>
-where service is one of:
-  zookeeper                   Run the zookeeper server
-  bookie                      Run the bookie server
-  bookie-rereplicator         Run the bookie rereplicator
-  writeproxy                  Run the write proxy server
-  writeproxy-monitor          Run the write proxy monitor
-
-where argument is one of:
-  -force (accepted only with stop service): Decides whether to stop the process forcefully if not stopped by normal shutdown
-EOF
-}
-
-BINDIR=$(dirname "$0")
-DL_HOME=$(cd $BINDIR/..;pwd)
-
-if [ -f "${DL_HOME}/conf/dlogenv.sh" ]; then
- . "${DL_HOME}/conf/dlogenv.sh"
-fi
-
-SERVICE_PORT=${SERVICE_PORT:-"0"}
-SERVICE_ARGS=""
-
-# DLOG logging configuration
-DLOG_LOG_DIR=${DLOG_LOG_DIR:-"$DL_HOME/logs"}
-DLOG_ROOT_LOGGER=${DLOG_ROOT_LOGGER:-'INFO,R'}
-
-# Process Control Parameters
-DLOG_STOP_TIMEOUT=${DLOG_STOP_TIMEOUT:-30}
-DLOG_PID_DIR=${DLOG_PID_DIR:-$DL_HOME/pids}
-
-if [ $# -lt 2 ]; then
-  echo "Error: not enough arguments provided."
-  usage
-  exit 1
-fi
-
-command=$1
-shift
-
-service=$1
-shift
-
-service_class=$service
-case $service in
-  (zookeeper)
-    service_class="org.apache.zookeeper.server.quorum.QuorumPeerMain"
-    DLOG_ROOT_LOGGER=${ZK_ROOT_LOGGER:-'INFO,R'}
-    ;;
-  (bookie)
-    service_class="org.apache.bookkeeper.proto.BookieServer"
-    DLOG_ROOT_LOGGER=${BK_ROOT_LOGGER:-'INFO,R'}
-    ;;
-  (bookie-rereplicator)
-    service_class="org.apache.bookkeeper.replication.AutoRecoveryMain"
-    DLOG_ROOT_LOGGER=${BK_ROOT_LOGGER:-'INFO,R'}
-    ;;
-  (writeproxy)
-    service_class="org.apache.distributedlog.service.DistributedLogServerApp"
-    DLOG_ROOT_LOGGER=${WP_ROOT_LOGGER:-'INFO,R'}
-    WP_CONF_FILE=${WP_CONF_FILE:-"$DL_HOME/conf/write_proxy.conf"}
-    WP_SERVICE_PORT=${WP_SERVICE_PORT:-'4181'}
-    WP_STATS_PORT=${WP_STATS_PORT:-'9000'}
-    WP_STATS_PROVIDER=${WP_STATS_PROVIDER:-'org.apache.bookkeeper.stats.CodahaleMetricsServletProvider'}
-    WP_SHARD_ID=${WP_SHARD_ID:-'0'}
-    WP_NAMESPACE=${WP_NAMESPACE:-'distributedlog://127.0.0.1:2181/messaging/distributedlog/mynamespace'}
-    SERVICE_PORT=${WP_SERVICE_PORT}
-    SERVICE_ARGS="--conf ${WP_CONF_FILE} --uri ${WP_NAMESPACE} --shard-id ${WP_SHARD_ID} --port ${WP_SERVICE_PORT} --stats-port ${WP_STATS_PORT} --stats-provider ${WP_STATS_PROVIDER} --announce --thriftmux"
-    DLOG_EXTRA_OPTS="${DLOG_EXTRA_OPTS} -DcodahaleStatsHttpPort=${WP_STATS_PORT} -Dserver_port=${WP_SERVICE_PORT} -Dserver_shard=${WP_SHARD_ID}"
-    ;;
-  (writeproxy-monitor)
-    ;;
-  (*)
-    echo "Error: unknown service name $service"
-    usage
-    exit 1
-    ;;
-esac
-
-echo "doing $command $service ..."
-
-export DLOG_LOG_DIR=$DLOG_LOG_DIR
-export DLOG_ROOT_LOGGER=$DLOG_ROOT_LOGGER
-export DLOG_LOG_FILE=dlog-$service-$HOSTNAME-$SERVICE_PORT.log
-export DLOG_EXTRA_OPTS=$DLOG_EXTRA_OPTS
-
-pid=$DLOG_PID_DIR/dlog-$service-$SERVICE_PORT.pid
-out=$DLOG_LOG_DIR/dlog-$service-$HOSTNAME-$SERVICE_PORT.out
-logfile=$DLOG_LOG_DIR/$DLOG_LOG_FILE
-
-rotate_out_log () {
-  log=$1;
-  num=5;
-  if [ -n "$2" ]; then
-   num=$2
-  fi
-  if [ -f "$log" ]; then # rotate logs
-    while [ $num -gt 1 ]; do
-      prev=`expr $num - 1`
-      [ -f "$log.$prev" ] && mv "$log.$prev" "$log.$num"
-      num=$prev
-    done
-    mv "$log" "$log.$num";
-  fi
-}
-
-mkdir -p "$DLOG_LOG_DIR"
-mkdir -p "$DLOG_PID_DIR"
-
-case $command in
-  (start)
-    if [ -f $pid ]; then
-      if kill -0 $(cat $pid) > /dev/null 2>&1; then
-        echo $service running as process $(cat $pid).  Stop it first.
-        exit 1
-      fi
-    fi
-
-    rotate_out_log $out
-    echo starting $service, logging to $logfile
-    dlog=$DL_HOME/bin/dlog
-    nohup $dlog $service_class ${SERVICE_ARGS} "$@" > "$out" 2>&1 < /dev/null &
-    echo $! > $pid
-    sleep 1; head $out
-    sleep 2;
-    if ! ps -p $! > /dev/null ; then
-      exit 1
-    fi
-    ;;
-
-  (stop)
-    if [ -f $pid ]; then
-      TARGET_PID=$(cat $pid)
-      if kill -0 $TARGET_PID > /dev/null 2>&1; then
-        echo stopping $service
-        kill $TARGET_PID
-
-        count=0
-        location=$DLOG_LOG_DIR
-        while ps -p $TARGET_PID > /dev/null; do
-          echo "Shutdown is in progress... Please wait..."
-          sleep 1
-          count=$(expr $count + 1)
-          if [ "$count" = "$DLOG_STOP_TIMEOUT" ]; then
-            break
-          fi
-        done
-        if [ "$count" != "$DLOG_STOP_TIMEOUT" ]; then
-          echo "Shutdown completed."
-        fi
-        if kill -0 $TARGET_PID > /dev/null 2>&1; then
-          fileName=$location/$service.out
-          $JAVA_HOME/bin/jstack $TARGET_PID > $fileName
-          echo Thread dumps are taken for analysis at $fileName
-          if [ "$1" == "-force" ]; then
-            echo forcefully stopping $service
-            kill -9 $TARGET_PID >/dev/null 2>&1
-            echo Successfully stopped the process
-          else
-            echo "WARNNING : $service is not stopped completely."
-            exit 1
-          fi
-        fi
-      else
-        echo no $service to stop
-      fi
-      rm $pid
-    else
-      echo no $service to stop
-    fi
-    ;;
-
-  (*)
-    usage
-    echo $supportedargs
-    exit 1
-    ;;
-esac

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/bin/dlog-env.sh
----------------------------------------------------------------------
diff --git a/distributedlog-service/bin/dlog-env.sh b/distributedlog-service/bin/dlog-env.sh
deleted file mode 100644
index 50a1960..0000000
--- a/distributedlog-service/bin/dlog-env.sh
+++ /dev/null
@@ -1,35 +0,0 @@
-#!/usr/bin/env bash
-#
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-# we need the DLog URI to be set
-if [[ -z "${DISTRIBUTEDLOG_URI}" ]]; then
-  echo "Environment variable DISTRIBUTEDLOG_URI is not set."
-  exit 1
-fi
-
-# add the jars from current dir to the class path (should be distributedlog-service)
-for i in ./*.jar; do
-  CLASSPATH="$i:${CLASSPATH}"
-done
-
-# add all the jar from lib/ to the class path
-for i in ./lib/*.jar; do
-  CLASSPATH="$i:${CLASSPATH}"
-done

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/bin/dlog-start.sh
----------------------------------------------------------------------
diff --git a/distributedlog-service/bin/dlog-start.sh b/distributedlog-service/bin/dlog-start.sh
deleted file mode 100755
index c9c2192..0000000
--- a/distributedlog-service/bin/dlog-start.sh
+++ /dev/null
@@ -1,34 +0,0 @@
-#!/bin/bash
-#
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-DLOGBIN="${BASH_SOURCE-$0}"
-DLOGBIN="$(dirname "${DLOGBIN}")"
-DLOGBINDIR="$(cd "${DLOGBIN}"; pwd)"
-
-. "${DLOGBINDIR}"/dlog-env.sh
-
-java -cp "${CLASSPATH}" \
-     -Dlog4j.configuration=conf/log4j.properties \
-     -DstatsHttpPort=9000 -DstatsExport=true \
-     -Dserver_shard=0 \
-     org.apache.distributedlog.service.DistributedLogServerApp \
-     --port 8000 \
-     --uri "${DISTRIBUTEDLOG_URI}" \
-     --conf conf/distributedlog.conf

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/bin/zk-server-start.sh
----------------------------------------------------------------------
diff --git a/distributedlog-service/bin/zk-server-start.sh b/distributedlog-service/bin/zk-server-start.sh
deleted file mode 100644
index f222756..0000000
--- a/distributedlog-service/bin/zk-server-start.sh
+++ /dev/null
@@ -1,44 +0,0 @@
-#!/bin/bash
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-# 
-#    http://www.apache.org/licenses/LICENSE-2.0
-# 
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-if [ $# -lt 1 ];
-then
-    echo "USAGE: $0 [-daemon] zookeeper.properties"
-    exit 1
-fi
-BASE_DIR=$(dirname $0)
-
-if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then
-    export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$BASE_DIR/../config/log4j.properties"
-fi
-
-if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then
-    export KAFKA_HEAP_OPTS="-Xmx512M -Xms512M"
-fi
-
-EXTRA_ARGS="-name zookeeper -loggc"
-
-COMMAND=$1
-case $COMMAND in
-  -daemon)
-     EXTRA_ARGS="-daemon "$EXTRA_ARGS
-     shift
-     ;;
- *)
-     ;;
-esac
-
-exec $BASE_DIR/kafka-run-class.sh $EXTRA_ARGS org.apache.zookeeper.server.quorum.QuorumPeerMain "$@"

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/conf/bookie.conf.template
----------------------------------------------------------------------
diff --git a/distributedlog-service/conf/bookie.conf.template b/distributedlog-service/conf/bookie.conf.template
deleted file mode 100644
index 5ca89d0..0000000
--- a/distributedlog-service/conf/bookie.conf.template
+++ /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.
-# */
-
-## Bookie settings
-
-# Port that bookie server listen on
-bookiePort=3181
-
-# TODO: change the journal directory
-# Directory Bookkeeper outputs its write ahead log
-journalDirectory=/tmp/data/bk/journal
-
-# TODO: change the ledgers directory
-# Directory Bookkeeper outputs ledger snapshots
-ledgerDirectories=/tmp/data/bk/ledgers
-
-# TODO: change the index directory
-# Directory in which index files will be stored.
-indexDirectories=/tmp/data/bk/ledgers
-
-# Ledger Manager Class
-# What kind of ledger manager is used to manage how ledgers are stored, managed
-# and garbage collected. Try to read 'BookKeeper Internals' for detail info.
-ledgerManagerType=hierarchical
-
-# Root zookeeper path to store ledger metadata
-# This parameter is used by zookeeper-based ledger manager as a root znode to
-# store all ledgers.
-zkLedgersRootPath=/messaging/bookkeeper/ledgers
-
-# Max file size of entry logger, in bytes
-# A new entry log file will be created when the old one reaches the file size limitation
-logSizeLimit=1073741823
-
-# Max file size of journal file, in mega bytes
-# A new journal file will be created when the old one reaches the file size limitation
-#
-journalMaxSizeMB=2048
-
-# Max number of old journal file to kept
-# Keep a number of old journal files would help data recovery in specia case
-#
-journalMaxBackups=5
-
-# How long the interval to trigger next garbage collection, in milliseconds
-# Since garbage collection is running in background, too frequent gc
-# will heart performance. It is better to give a higher number of gc
-# interval if there is enough disk capacity.
-# gc per 1 hour (aligning with most DL rolling interval)
-gcInitialWaitTime=600000
-gcWaitTime=3600000
-# do minor compaction per 2 hours
-minorCompactionInterval=7200
-minorCompactionThreshold=0.2
-# disable major compaction
-majorCompactionInterval=0
-# reduce major compaction threshold to a low value to prevent bad force compaction behavior
-majorCompactionThreshold=0.3
-# Compaction Rate & Max Outstanding
-compactionRate=10737418
-compactionMaxOutstandingRequests=10737418
-
-# How long the interval to flush ledger index pages to disk, in milliseconds
-# Flushing index files will introduce much random disk I/O.
-# If separating journal dir and ledger dirs each on different devices,
-# flushing would not affect performance. But if putting journal dir
-# and ledger dirs on same device, performance degrade significantly
-# on too frequent flushing. You can consider increment flush interval
-# to get better performance, but you need to pay more time on bookie
-# server restart after failure.
-#
-flushInterval=1000
-
-# Interval to watch whether bookie is dead or not, in milliseconds
-#
-# bookieDeathWatchInterval=1000
-
-## zookeeper client settings
-
-# A list of one of more servers on which zookeeper is running.
-# The server list can be comma separated values, for example:
-# zkServers=zk1:2181,zk2:2181,zk3:2181
-zkServers=localhost:2181
-
-# ZooKeeper client session timeout in milliseconds
-# Bookie server will exit if it received SESSION_EXPIRED because it
-# was partitioned off from ZooKeeper for more than the session timeout
-# JVM garbage collection, disk I/O will cause SESSION_EXPIRED.
-# Increment this value could help avoiding this issue
-zkTimeout=30000
-
-## NIO Server settings
-
-# This settings is used to enabled/disabled Nagle's algorithm, which is a means of
-# improving the efficiency of TCP/IP networks by reducing the number of packets
-# that need to be sent over the network.
-# If you are sending many small messages, such that more than one can fit in
-# a single IP packet, setting server.tcpnodelay to false to enable Nagle algorithm
-# can provide better performance.
-# Default value is true.
-#
-serverTcpNoDelay=true
-
-## ledger cache settings
-
-# Max number of ledger index files could be opened in bookie server
-# If number of ledger index files reaches this limitation, bookie
-# server started to swap some ledgers from memory to disk.
-# Too frequent swap will affect performance. You can tune this number
-# to gain performance according your requirements.
-openFileLimit=20000
-
-# Size of a index page in ledger cache, in bytes
-# A larger index page can improve performance writing page to disk,
-# which is efficent when you have small number of ledgers and these
-# ledgers have similar number of entries.
-# If you have large number of ledgers and each ledger has fewer entries,
-# smaller index page would improve memory usage.
-pageSize=8192
-
-# How many index pages provided in ledger cache
-# If number of index pages reaches this limitation, bookie server
-# starts to swap some ledgers from memory to disk. You can increment
-# this value when you found swap became more frequent. But make sure
-# pageLimit*pageSize should not more than JVM max memory limitation,
-# otherwise you would got OutOfMemoryException.
-# In general, incrementing pageLimit, using smaller index page would
-# gain bettern performance in lager number of ledgers with fewer entries case
-# If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute
-# the limitation of number of index pages.
-pageLimit=131072
-
-#If all ledger directories configured are full, then support only read requests for clients.
-#If "readOnlyModeEnabled=true" then on all ledger disks full, bookie will be converted
-#to read-only mode and serve only read requests. Otherwise the bookie will be shutdown.
-readOnlyModeEnabled=true
-
-# Bookie Journal Settings
-writeBufferSizeBytes=262144
-journalFlushWhenQueueEmpty=false
-journalRemoveFromPageCache=true
-journalAdaptiveGroupWrites=true
-journalMaxGroupWaitMSec=4
-journalBufferedEntriesThreshold=180
-journalBufferedWritesThreshold=131072
-journalMaxGroupedEntriesToCommit=200
-journalPreAllocSizeMB=4
-
-# Sorted Ledger Storage Settings
-sortedLedgerStorageEnabled=true
-skipListSizeLimit=67108864
-skipListArenaChunkSize=2097152
-skipListArenaMaxAllocSize=131072
-fileInfoCacheInitialCapacity=10000
-fileInfoMaxIdleTime=3600
-
-# Bookie Threads Settings (NOTE: change this to align the cpu cores)
-numAddWorkerThreads=4
-numJournalCallbackThreads=4
-numReadWorkerThreads=4
-numLongPollWorkerThreads=4
-
-# stats
-statsProviderClass=org.apache.bookkeeper.stats.CodahaleMetricsServletProvider
-# Exporting codahale stats
-codahaleStatsHttpPort=9001
-useHostNameAsBookieID=true
-allowLoopback=true

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/conf/distributedlog.conf
----------------------------------------------------------------------
diff --git a/distributedlog-service/conf/distributedlog.conf b/distributedlog-service/conf/distributedlog.conf
deleted file mode 100644
index dac71ac..0000000
--- a/distributedlog-service/conf/distributedlog.conf
+++ /dev/null
@@ -1,125 +0,0 @@
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-########################
-# ZooKeeper Client Settings
-########################
-
-# zookeeper settings
-zkSessionTimeoutSeconds=30
-zkNumRetries=0
-zkRetryStartBackoffMillis=100
-zkRetryMaxBackoffMillis=200
-# bkc zookeeper settings
-bkcZKSessionTimeoutSeconds=60
-bkcZKNumRetries=20
-bkcZKRetryStartBackoffMillis=100
-bkcZKRetryMaxBackoffMillis=200
-
-########################
-# BookKeeper Client Settings
-########################
-
-# bookkeeper client timeouts
-bkcWriteTimeoutSeconds=10
-bkcReadTimeoutSeconds=1
-bkcNumWorkerThreads=16
-# bkcNumIOThreads=16
-bkc.numChannelsPerBookie=1
-bkc.enableTaskExecutionStats=true
-bkc.connectTimeoutMillis=1000
-bkc.enablePerHostStats=true
-
-########################
-# DL Settings
-########################
-
-# lock timeout
-lockTimeoutSeconds=0
-# dl worker threads
-numWorkerThreads=16
-
-### Recovery Related Settings
-
-# recover log segments in background
-recoverLogSegmentsInBackground=true
-# disable max id in proxy
-maxIdSanityCheck=true
-# use allocator pool for proxy
-enableLedgerAllocatorPool=false
-# ledger allocator pool size
-ledgerAllocatorPoolCoreSize=20
-# check stream exists or not
-createStreamIfNotExists=true
-# encode dc id in version
-encodeDCIDInVersion=true
-# logSegmentNameVersion
-logSegmentNameVersion=1
-
-### Write Performance Related Settings
-
-# ensemble size
-ensemble-size=3
-write-quorum-size=3
-ack-quorum-size=2
-bkc.ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
-bkc.delayEnsembleChange=true
-
-# sync settings
-# buffer size is large because when we rewrite we perform a very large write to persist
-# all queue state at once (up to max queue memory size, ex. 16MB). the write will be
-# throttled if it takes too long, which can hurt performance, so important to optimize
-# for this case.
-output-buffer-size=512000
-enableImmediateFlush=false
-periodicFlushFrequencyMilliSeconds=6
-logFlushTimeoutSeconds=120
-
-### Ledger Rolling Related Settings
-
-# retention policy
-retention-size=0
-# rolling ledgers (disable time rolling/enable size rolling)
-rolling-interval=0
-
-# max logsegment bytes=2GB
-# much larger than max journal size, effectively never roll and let drpc do it
-maxLogSegmentBytes=2147483648
-
-# rolling concurrency
-logSegmentRollingConcurrency=1
-# disable sanityCheckDelete
-sanityCheckDelete=false
-ledgerAllocatorPoolName=drpc-alloc-pool
-
-### Readahead settings
-
-enableReadAhead=true
-ReadAheadBatchSize=10
-ReadAheadMaxEntries=100
-ReadAheadWaitTime=10
-
-### Rate limit
-
-rpsSoftWriteLimit=1
-rpsHardWriteLimit=5
-rpsHardServiceLimit=15
-
-### Config
-
-dynamicConfigReloadIntervalSec=5

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/conf/dlogenv.sh
----------------------------------------------------------------------
diff --git a/distributedlog-service/conf/dlogenv.sh b/distributedlog-service/conf/dlogenv.sh
deleted file mode 100644
index 345e60f..0000000
--- a/distributedlog-service/conf/dlogenv.sh
+++ /dev/null
@@ -1,75 +0,0 @@
-#!/bin/sh
-#
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-##################
-# General
-##################
-
-# Log4j configuration file
-# DLOG_LOG_CONF=
-
-# Extra options to be passed to the jvm
-# DLOG_EXTRA_OPTS=
-
-# Add extra paths to the dlog classpath
-# DLOG_EXTRA_CLASSPATH=
-
-# Configure the root logger
-# DLOG_ROOT_LOGGER=
-
-# Configure the log dir
-# DLOG_LOG_DIR=
-
-# Configure the log file
-# DLOG_LOG_FILE=
-
-#################
-# ZooKeeper
-#################
-
-# Configure zookeeper root logger
-# ZK_ROOT_LOGGER=
-
-#################
-# Bookie
-#################
-
-# Configure bookie root logger
-# BK_ROOT_LOGGER=
-
-#################
-# Write Proxy
-#################
-
-# Configure write proxy root logger
-# WP_ROOT_LOGGER=
-
-# write proxy configuration file
-# WP_CONF_FILE=${DL_HOME}/conf/write_proxy.conf
-
-# port and stats port
-# WP_SERVICE_PORT=4181
-# WP_STATS_PORT=9000
-
-# shard id
-# WP_SHARD_ID=0
-
-# write proxy namespace
-# WP_NAMESPACE=distributedlog://127.0.0.1:2181/messaging/distributedlog/mynamespace

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-service/conf/log4j.properties b/distributedlog-service/conf/log4j.properties
deleted file mode 100644
index 73b4cfa..0000000
--- a/distributedlog-service/conf/log4j.properties
+++ /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.
-# */
-
-#
-# DistributedLog Logging Configuration
-#
-
-# Default values
-dlog.root.logger=INFO, R
-dlog.log.dir=logs
-dlog.log.file=dlog.log
-
-log4j.rootLogger=${dlog.root.logger}
-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.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.bookkeeper.util.SafeRunnable=false
-
-log4j.appender.Executors=org.apache.log4j.RollingFileAppender
-log4j.appender.Executors.Threshold=INFO
-log4j.appender.Executors.File=${dlog.log.dir}/executors.log
-log4j.appender.Executors.MaxFileSize=20MB
-log4j.appender.Executors.MaxBackupIndex=5
-log4j.appender.Executors.layout=org.apache.log4j.PatternLayout
-log4j.appender.Executors.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-log4j.appender.R=org.apache.log4j.RollingFileAppender
-log4j.appender.R.Threshold=INFO
-log4j.appender.R.File=${dlog.log.dir}/${dlog.log.file}
-log4j.appender.R.MaxFileSize=20MB
-log4j.appender.R.MaxBackupIndex=50
-log4j.appender.R.layout=org.apache.log4j.PatternLayout
-log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-log4j.appender.stderr=org.apache.log4j.ConsoleAppender
-log4j.appender.stderr.Target=System.err
-log4j.appender.stderr.Threshold=INFO
-log4j.appender.stderr.layout=org.apache.log4j.PatternLayout
-log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/conf/write_proxy.conf
----------------------------------------------------------------------
diff --git a/distributedlog-service/conf/write_proxy.conf b/distributedlog-service/conf/write_proxy.conf
deleted file mode 100644
index 7f5351a..0000000
--- a/distributedlog-service/conf/write_proxy.conf
+++ /dev/null
@@ -1,143 +0,0 @@
-#/**
-# * Licensed to the Apache Software Foundation (ASF) under one
-# * or more contributor license agreements.  See the NOTICE file
-# * distributed with this work for additional information
-# * regarding copyright ownership.  The ASF licenses this file
-# * to you under the Apache License, Version 2.0 (the
-# * "License"); you may not use this file except in compliance
-# * with the License.  You may obtain a copy of the License at
-# *
-# *     http://www.apache.org/licenses/LICENSE-2.0
-# *
-# * Unless required by applicable law or agreed to in writing, software
-# * distributed under the License is distributed on an "AS IS" BASIS,
-# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# * See the License for the specific language governing permissions and
-# * limitations under the License.
-# */
-
-########################
-# ZooKeeper Client Settings
-########################
-
-# zookeeper settings
-zkSessionTimeoutSeconds=1
-zkNumRetries=0
-zkRetryStartBackoffMillis=100
-zkRetryMaxBackoffMillis=200
-# bkc zookeeper settings
-bkcZKSessionTimeoutSeconds=60
-bkcZKNumRetries=20
-bkcZKRetryStartBackoffMillis=100
-bkcZKRetryMaxBackoffMillis=200
-
-########################
-# BookKeeper Client Settings
-########################
-
-# bookkeeper client timeouts
-bkcWriteTimeoutSeconds=2
-bkcReadTimeoutSeconds=2
-bkcNumWorkerThreads=32
-bkc.numChannelsPerBookie=1
-bkc.enableTaskExecutionStats=true
-bkc.connectTimeoutMillis=200
-bkc.enableParallelRecoveryRead=true
-bkc.recoveryReadBatchSize=5
-bkc.enablePerHostStats=true
-
-########################
-# DL Settings
-########################
-
-# Metadata Settings
-
-# ledger metadata version that supports sequence id
-ledger-metadata-layout=5
-
-# lock timeout
-lockTimeoutSeconds=0
-# dl worker threads
-numWorkerThreads=32
-
-### Recovery Related Settings
-
-# recover log segments in background
-recoverLogSegmentsInBackground=false
-# disable max id in proxy
-maxIdSanityCheck=false
-# use allocator pool for proxy
-enableLedgerAllocatorPool=true
-# ledger allocator pool path
-ledgerAllocatorPoolPath=.write_proxy_allocation_pool
-# ledger allocator pool size
-ledgerAllocatorPoolCoreSize=40
-# check stream exists or not
-createStreamIfNotExists=true
-# encode dc id in version
-encodeDCIDInVersion=true
-# logSegmentNameVersion
-logSegmentNameVersion=1
-
-### Write Performance Related Settings
-
-# ensemble size
-ensemble-size=3
-write-quorum-size=3
-ack-quorum-size=2
-bkc.ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
-bkc.delayEnsembleChange=true
-bkc.writeRequestToChannelAsync=true
-
-# enable immediate flush
-enableImmediateFlush=true
-# 0k output buffer
-output-buffer-size=0
-# disable periodical flush
-periodicFlushFrequencyMilliSeconds=0
-enableTaskExecutionStats=true
-taskExecutionWarnTimeMicros=100000
-
-### Ledger Rolling Related Settings
-
-# retention policy
-retention-size=4
-# rolling ledgers (enable time rolling): 120 minutes = 2 hours
-rolling-interval=120
-# max logsegment bytes : 2GB
-maxLogSegmentBytes=2147483648
-# rolling concurrency
-logSegmentRollingConcurrency=1
-# disable sanityCheckDelete
-sanityCheckDelete=false
-# compression codec
-compressionType=lz4
-
-### Per Stream Stats
-enablePerStreamStat=true
-
-########################
-# DL Settings
-########################
-
-# proxy server settings
-server_mode=DURABLE
-serviceTimeoutMs=60000
-streamProbationTimeoutMs=120000
-server_threads=16
-server_dlsn_version=1
-server_enable_perstream_stat=true
-server_graceful_shutdown_period_ms=20000
-
-# write limits
-perWriterOutstandingWriteLimit=-1
-globalOutstandingWriteLimit=-1
-outstandingWriteLimitDarkmode=false
-
-# bytes per second limit applied at the host level (50MBps on 1Gib machines)
-bpsHardServiceLimit=52428800
-# bytes per second limit after which no new streams may be acquired (65MBps on 1Gib machines)
-bpsStreamAcquireServiceLimit=47185920
-
-# limit the maximum number of streams
-maxAcquiredPartitionsPerProxy=-1

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/conf/zookeeper.conf.template
----------------------------------------------------------------------
diff --git a/distributedlog-service/conf/zookeeper.conf.template b/distributedlog-service/conf/zookeeper.conf.template
deleted file mode 100644
index 3c0546e..0000000
--- a/distributedlog-service/conf/zookeeper.conf.template
+++ /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.
-
-# The number of milliseconds of each tick
-tickTime=2000
-
-# the port at which the clients will connect
-clientPort=2181
-
-# The number of ticks that the initial
-# synchronization phase can take
-initLimit=10
-
-# The number of ticks that can pass between
-# sending a request and getting an acknowledgement
-syncLimit=30
-
-# the directory where the snapshot is stored.
-dataDir=/tmp/data/zookeeper
-
-# where txlog  are written
-dataLogDir=/tmp/data/zookeeper/txlog
-
-# the port at which the admin will listen
-adminPort=9990
-zookeeper.admin.enableServer=true
-
-# limit on queued clients - default: 1000
-globalOutstandingLimit=1000
-
-# number of transactions before snapshots are taken - default: 100000
-snapCount=100000
-
-# max # of clients - 0==unlimited
-maxClientCnxns=25
-
-# Election implementation to use. A value of "0" corresponds to the original
-# UDP-based version, "1" corresponds to the non-authenticated UDP-based
-# version of fast leader election, "2" corresponds to the authenticated
-# UDP-based version of fast leader election, and "3" corresponds to TCP-based
-# version of fast leader election. Currently, only 0 and 3 are supported,
-# 3 being the default
-electionAlg=3
-
-# Leader accepts client connections. Default value is "yes". The leader
-# machine coordinates updates. For higher update throughput at thes slight
-# expense of read throughput the leader can be configured to not accept
-# clients and focus on coordination.
-leaderServes=yes
-
-# Skips ACL checks. This results in a boost in throughput, but opens up full
-# access to the data tree to everyone.
-skipACL=no
-
-# Purge txn logs every hour. Before 3.4.x this was done with an external cron
-# job, now we can do it internally.
-autopurge.purgeInterval=1
-
-# Prior to version 3.4 ZooKeeper has always used NIO directly, however in
-# versions 3.4 and later Netty is supported as an option to NIO (replaces).
-# serverCnxnFactory=org.apache.zookeeper.server.NIOServerCnxnFactory
-
-standaloneEnabled=false
-# ZooKeeper Dynamic Reconfiguration
-# See: https://zookeeper.apache.org/doc/trunk/zookeeperReconfig.html
-#
-# standaloneEnabled=false
-# dynamicConfigFile=/path/to/zoo.cfg.dynamic
-#
-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/c44e0278/distributedlog-service/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-service/pom.xml b/distributedlog-service/pom.xml
deleted file mode 100644
index eaccb6b..0000000
--- a/distributedlog-service/pom.xml
+++ /dev/null
@@ -1,270 +0,0 @@
-<?xml version="1.0"?>
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.distributedlog</groupId>
-    <artifactId>distributedlog</artifactId>
-    <version>0.5.0-incubating-SNAPSHOT</version>
-  </parent>
-  <artifactId>distributedlog-service</artifactId>
-  <name>Apache DistributedLog :: Proxy Service</name>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-client</artifactId>
-      <version>${project.parent.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.zookeeper</groupId>
-          <artifactId>zookeeper</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-core</artifactId>
-      <version>${project.parent.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>finagle-ostrich4_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>finagle-thriftmux_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-core</artifactId>
-      <version>${project.parent.version}</version>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
-      <version>${zookeeper.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-log4j12</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${junit.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <version>${mockito.version}</version>
-      <scope>test</scope>
-    </dependency> 
-    <dependency>
-      <groupId>org.apache.bookkeeper.stats</groupId>
-      <artifactId>codahale-metrics-provider</artifactId>
-      <version>${bookkeeper.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-servlets</artifactId>
-      <version>${codahale.metrics.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
-      <version>${jetty.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlet</artifactId>
-      <version>${jetty.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
-      <version>${libthrift.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>scrooge-core_2.11</artifactId>
-      <version>${scrooge.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <version>${slf4j.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.curator</groupId>
-      <artifactId>curator-test</artifactId>
-      <version>${curator.version}</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.zookeeper</groupId>
-          <artifactId>zookeeper</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-protocol</artifactId>
-      <version>${project.parent.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>com.twitter</groupId>
-        <artifactId>scrooge-maven-plugin</artifactId>
-        <version>${scrooge-maven-plugin.version}</version>
-        <configuration>
-          <language>java</language>
-        </configuration>
-        <executions>
-          <execution>
-            <id>thrift-sources</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>compile</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <artifactId>maven-assembly-plugin</artifactId>
-        <version>${maven-assembly-plugin.version}</version>
-        <configuration>
-          <descriptors>
-            <descriptor>../src/assemble/bin.xml</descriptor>
-          </descriptors>
-        </configuration>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>single</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <version>${maven-surefire-plugin.version}</version>
-        <configuration>
-          <redirectTestOutputToFile>true</redirectTestOutputToFile>
-          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
-          <forkMode>always</forkMode>
-          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.distributedlog.TimedOutTestsListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-        <configuration>
-          <excludeFilterFile>${basedir}/src/main/resources/findbugsExclude.xml</excludeFilterFile>
-        </configuration>
-      </plugin>
-      <plugin>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>copy-dependencies</goal>
-            </goals>
-            <configuration>
-              <outputDirectory>${basedir}/lib</outputDirectory>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-        <version>${maven-checkstyle-plugin.version}</version>
-        <dependencies>
-          <dependency>
-            <groupId>com.puppycrawl.tools</groupId>
-            <artifactId>checkstyle</artifactId>
-            <version>${puppycrawl.checkstyle.version}</version>
-          </dependency>
-          <dependency>
-            <groupId>org.apache.distributedlog</groupId>
-            <artifactId>distributedlog-build-tools</artifactId>
-            <version>${project.version}</version>
-          </dependency>
-        </dependencies>
-        <configuration>
-          <configLocation>distributedlog/checkstyle.xml</configLocation>
-          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
-          <consoleOutput>true</consoleOutput>
-          <failOnViolation>true</failOnViolation>
-          <includeResources>false</includeResources>
-          <includeTestSourceDirectory>true</includeTestSourceDirectory>
-        </configuration>
-        <executions>
-          <execution>
-            <phase>test-compile</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-  <profiles>
-    <profile>
-      <id>twitter-ostrich-provider</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.bookkeeper.stats</groupId>
-          <artifactId>twitter-ostrich-provider</artifactId>
-          <version>${bookkeeper.version}</version>
-          <exclusions>
-            <exclusion>
-              <groupId>com.twitter</groupId>
-              <artifactId>ostrich_2.10</artifactId>
-            </exclusion>
-            <exclusion>
-              <groupId>com.twitter</groupId>
-              <artifactId>ostrich_2.9.2</artifactId>
-            </exclusion>
-          </exclusions>
-        </dependency>
-      </dependencies>
-    </profile>
-  </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.java b/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.java
deleted file mode 100644
index 8db3e90..0000000
--- a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/CodahaleMetricsServletProvider.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 org.apache.bookkeeper.stats;
-
-import com.codahale.metrics.health.HealthCheckRegistry;
-import org.apache.commons.configuration.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Extend the codahale metrics provider to run servlets.
- */
-public class CodahaleMetricsServletProvider extends CodahaleMetricsProvider {
-
-    private static final Logger logger = LoggerFactory.getLogger(CodahaleMetricsServletProvider.class);
-
-    ServletReporter servletReporter = null;
-    private final HealthCheckRegistry healthCheckRegistry = new HealthCheckRegistry();
-
-    @Override
-    public void start(Configuration conf) {
-        super.start(conf);
-        Integer httpPort = conf.getInteger("codahaleStatsHttpPort", null);
-        if (null != httpPort) {
-            servletReporter = new ServletReporter(
-                    getMetrics(),
-                    healthCheckRegistry,
-                    httpPort);
-            try {
-                servletReporter.start();
-            } catch (Exception e) {
-                logger.warn("Encountered error on starting the codahale metrics servlet", e);
-            }
-        }
-    }
-
-    @Override
-    public void stop() {
-        if (null != servletReporter) {
-            try {
-                servletReporter.stop();
-            } catch (Exception e) {
-                logger.error("Encountered error on stopping the codahale metrics servlet", e);
-            }
-        }
-        super.stop();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.java b/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.java
deleted file mode 100644
index 348787a..0000000
--- a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/HealthCheckServletContextListener.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 org.apache.bookkeeper.stats;
-
-import com.codahale.metrics.health.HealthCheckRegistry;
-import com.codahale.metrics.servlets.HealthCheckServlet;
-
-/**
- * Health Check Servlet Listener.
- */
-public class HealthCheckServletContextListener extends HealthCheckServlet.ContextListener {
-
-    private final HealthCheckRegistry healthCheckRegistry;
-
-    public HealthCheckServletContextListener(HealthCheckRegistry healthCheckRegistry) {
-        this.healthCheckRegistry = healthCheckRegistry;
-    }
-
-    @Override
-    protected HealthCheckRegistry getHealthCheckRegistry() {
-        return healthCheckRegistry;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.java b/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.java
deleted file mode 100644
index 15279fe..0000000
--- a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/MetricsServletContextListener.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 org.apache.bookkeeper.stats;
-
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.servlets.MetricsServlet;
-
-/**
- * A servlet to report metrics.
- */
-public class MetricsServletContextListener extends MetricsServlet.ContextListener {
-
-    private final MetricRegistry metricRegistry;
-
-    public MetricsServletContextListener(MetricRegistry metricRegistry) {
-        this.metricRegistry = metricRegistry;
-    }
-
-    @Override
-    protected MetricRegistry getMetricRegistry() {
-        return metricRegistry;
-    }
-}



[08/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorService.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorService.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorService.java
deleted file mode 100644
index b1e2879..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorService.java
+++ /dev/null
@@ -1,469 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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 com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Sets;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-import com.twitter.common.zookeeper.ServerSet;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.DistributedLogManager;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.callback.LogSegmentListener;
-import org.apache.distributedlog.callback.NamespaceListener;
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.distributedlog.client.serverset.DLZkServerSet;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.Stat;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import java.io.File;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.ArrayList;
-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.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Monitor Service.
- */
-public class MonitorService implements NamespaceListener {
-
-    private static final Logger logger = LoggerFactory.getLogger(MonitorService.class);
-
-    private DistributedLogNamespace dlNamespace = null;
-    private MonitorServiceClient dlClient = null;
-    private DLZkServerSet[] zkServerSets = null;
-    private final ScheduledExecutorService executorService =
-            Executors.newScheduledThreadPool(Runtime.getRuntime().availableProcessors());
-    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-    private final Map<String, StreamChecker> knownStreams = new HashMap<String, StreamChecker>();
-
-    // Settings
-    private int regionId = DistributedLogConstants.LOCAL_REGION_ID;
-    private int interval = 100;
-    private String streamRegex = null;
-    private boolean watchNamespaceChanges = false;
-    private boolean handshakeWithClientInfo = false;
-    private int heartbeatEveryChecks = 0;
-    private int instanceId = -1;
-    private int totalInstances = -1;
-    private boolean isThriftMux = false;
-
-    // Options
-    private final Optional<String> uriArg;
-    private final Optional<String> confFileArg;
-    private final Optional<String> serverSetArg;
-    private final Optional<Integer> intervalArg;
-    private final Optional<Integer> regionIdArg;
-    private final Optional<String> streamRegexArg;
-    private final Optional<Integer> instanceIdArg;
-    private final Optional<Integer> totalInstancesArg;
-    private final Optional<Integer> heartbeatEveryChecksArg;
-    private final Optional<Boolean> handshakeWithClientInfoArg;
-    private final Optional<Boolean> watchNamespaceChangesArg;
-    private final Optional<Boolean> isThriftMuxArg;
-
-    // Stats
-    private final StatsProvider statsProvider;
-    private final StatsReceiver statsReceiver;
-    private final StatsReceiver monitorReceiver;
-    private final Stat successStat;
-    private final Stat failureStat;
-    private final Gauge<Number> numOfStreamsGauge;
-    // Hash Function
-    private final HashFunction hashFunction = Hashing.md5();
-
-    class StreamChecker implements Runnable, FutureEventListener<Void>, LogSegmentListener {
-        private final String name;
-        private volatile boolean closed = false;
-        private volatile boolean checking = false;
-        private final Stopwatch stopwatch = Stopwatch.createUnstarted();
-        private DistributedLogManager dlm = null;
-        private int numChecks = 0;
-
-        StreamChecker(String name) {
-            this.name = name;
-        }
-
-        @Override
-        public void run() {
-            if (null == dlm) {
-                try {
-                    dlm = dlNamespace.openLog(name);
-                    dlm.registerListener(this);
-                } catch (IOException e) {
-                    if (null != dlm) {
-                        try {
-                            dlm.close();
-                        } catch (IOException e1) {
-                            logger.error("Failed to close dlm for {} : ", name, e1);
-                        }
-                        dlm = null;
-                    }
-                    executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
-                }
-            } else {
-                stopwatch.reset().start();
-                boolean sendHeartBeat;
-                if (heartbeatEveryChecks > 0) {
-                    synchronized (this) {
-                        ++numChecks;
-                        if (numChecks >= Integer.MAX_VALUE) {
-                            numChecks = 0;
-                        }
-                        sendHeartBeat = (numChecks % heartbeatEveryChecks) == 0;
-                    }
-                } else {
-                    sendHeartBeat = false;
-                }
-                if (sendHeartBeat) {
-                    dlClient.heartbeat(name).addEventListener(this);
-                } else {
-                    dlClient.check(name).addEventListener(this);
-                }
-            }
-        }
-
-        @Override
-        public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
-            if (segments.size() > 0 && segments.get(0).getRegionId() == regionId) {
-                if (!checking) {
-                    logger.info("Start checking stream {}.", name);
-                    checking = true;
-                    run();
-                }
-            } else {
-                if (checking) {
-                    logger.info("Stop checking stream {}.", name);
-                }
-            }
-        }
-
-        @Override
-        public void onLogStreamDeleted() {
-            logger.info("Stream {} is deleted", name);
-        }
-
-        @Override
-        public void onSuccess(Void value) {
-            successStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            scheduleCheck();
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            failureStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            scheduleCheck();
-        }
-
-        private void scheduleCheck() {
-            if (closed) {
-                return;
-            }
-            if (!checking) {
-                return;
-            }
-            try {
-                executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
-            } catch (RejectedExecutionException ree) {
-                logger.error("Failed to schedule checking stream {} in {} ms : ",
-                        new Object[] { name, interval, ree });
-            }
-        }
-
-        private void close() {
-            closed = true;
-            if (null != dlm) {
-                try {
-                    dlm.close();
-                } catch (IOException e) {
-                    logger.error("Failed to close dlm for {} : ", name, e);
-                }
-            }
-        }
-    }
-
-    MonitorService(Optional<String> uriArg,
-                   Optional<String> confFileArg,
-                   Optional<String> serverSetArg,
-                   Optional<Integer> intervalArg,
-                   Optional<Integer> regionIdArg,
-                   Optional<String> streamRegexArg,
-                   Optional<Integer> instanceIdArg,
-                   Optional<Integer> totalInstancesArg,
-                   Optional<Integer> heartbeatEveryChecksArg,
-                   Optional<Boolean> handshakeWithClientInfoArg,
-                   Optional<Boolean> watchNamespaceChangesArg,
-                   Optional<Boolean> isThriftMuxArg,
-                   StatsReceiver statsReceiver,
-                   StatsProvider statsProvider) {
-        // options
-        this.uriArg = uriArg;
-        this.confFileArg = confFileArg;
-        this.serverSetArg = serverSetArg;
-        this.intervalArg = intervalArg;
-        this.regionIdArg = regionIdArg;
-        this.streamRegexArg = streamRegexArg;
-        this.instanceIdArg = instanceIdArg;
-        this.totalInstancesArg = totalInstancesArg;
-        this.heartbeatEveryChecksArg = heartbeatEveryChecksArg;
-        this.handshakeWithClientInfoArg = handshakeWithClientInfoArg;
-        this.watchNamespaceChangesArg = watchNamespaceChangesArg;
-        this.isThriftMuxArg = isThriftMuxArg;
-
-        // Stats
-        this.statsReceiver = statsReceiver;
-        this.monitorReceiver = statsReceiver.scope("monitor");
-        this.successStat = monitorReceiver.stat0("success");
-        this.failureStat = monitorReceiver.stat0("failure");
-        this.statsProvider = statsProvider;
-        this.numOfStreamsGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return knownStreams.size();
-            }
-        };
-    }
-
-    public void runServer() throws IllegalArgumentException, IOException {
-        checkArgument(uriArg.isPresent(),
-                "No distributedlog uri provided.");
-        checkArgument(serverSetArg.isPresent(),
-                "No proxy server set provided.");
-        if (intervalArg.isPresent()) {
-            interval = intervalArg.get();
-        }
-        if (regionIdArg.isPresent()) {
-            regionId = regionIdArg.get();
-        }
-        if (streamRegexArg.isPresent()) {
-            streamRegex = streamRegexArg.get();
-        }
-        if (instanceIdArg.isPresent()) {
-            instanceId = instanceIdArg.get();
-        }
-        if (totalInstancesArg.isPresent()) {
-            totalInstances = totalInstancesArg.get();
-        }
-        if (heartbeatEveryChecksArg.isPresent()) {
-            heartbeatEveryChecks = heartbeatEveryChecksArg.get();
-        }
-        if (instanceId < 0 || totalInstances <= 0 || instanceId >= totalInstances) {
-            throw new IllegalArgumentException("Invalid instance id or total instances number.");
-        }
-        handshakeWithClientInfo = handshakeWithClientInfoArg.isPresent();
-        watchNamespaceChanges = watchNamespaceChangesArg.isPresent();
-        isThriftMux = isThriftMuxArg.isPresent();
-        URI uri = URI.create(uriArg.get());
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-        if (confFileArg.isPresent()) {
-            String configFile = confFileArg.get();
-            try {
-                dlConf.loadConf(new File(configFile).toURI().toURL());
-            } catch (ConfigurationException e) {
-                throw new IOException("Failed to load distributedlog configuration from " + configFile + ".");
-            } catch (MalformedURLException e) {
-                throw new IOException("Failed to load distributedlog configuration from malformed "
-                        + configFile + ".");
-            }
-        }
-        logger.info("Starting stats provider : {}.", statsProvider.getClass());
-        statsProvider.start(dlConf);
-        String[] serverSetPaths = StringUtils.split(serverSetArg.get(), ",");
-        if (serverSetPaths.length == 0) {
-            throw new IllegalArgumentException("Invalid serverset paths provided : " + serverSetArg.get());
-        }
-
-        ServerSet[] serverSets = createServerSets(serverSetPaths);
-        ServerSet local = serverSets[0];
-        ServerSet[] remotes  = new ServerSet[serverSets.length - 1];
-        System.arraycopy(serverSets, 1, remotes, 0, remotes.length);
-
-        ClientBuilder finagleClientBuilder = ClientBuilder.get()
-            .connectTimeout(Duration.fromSeconds(1))
-            .tcpConnectTimeout(Duration.fromSeconds(1))
-            .requestTimeout(Duration.fromSeconds(2))
-            .keepAlive(true)
-            .failFast(false);
-
-        if (!isThriftMux) {
-            finagleClientBuilder = finagleClientBuilder
-                .hostConnectionLimit(2)
-                .hostConnectionCoresize(2);
-        }
-
-        dlClient = DistributedLogClientBuilder.newBuilder()
-                .name("monitor")
-                .thriftmux(isThriftMux)
-                .clientId(ClientId$.MODULE$.apply("monitor"))
-                .redirectBackoffMaxMs(50)
-                .redirectBackoffStartMs(100)
-                .requestTimeoutMs(2000)
-                .maxRedirects(2)
-                .serverSets(local, remotes)
-                .streamNameRegex(streamRegex)
-                .handshakeWithClientInfo(handshakeWithClientInfo)
-                .clientBuilder(finagleClientBuilder)
-                .statsReceiver(monitorReceiver.scope("client"))
-                .buildMonitorClient();
-        runMonitor(dlConf, uri);
-    }
-
-    ServerSet[] createServerSets(String[] serverSetPaths) {
-        ServerSet[] serverSets = new ServerSet[serverSetPaths.length];
-        zkServerSets = new DLZkServerSet[serverSetPaths.length];
-        for (int i = 0; i < serverSetPaths.length; i++) {
-            String serverSetPath = serverSetPaths[i];
-            zkServerSets[i] = parseServerSet(serverSetPath);
-            serverSets[i] = zkServerSets[i].getServerSet();
-        }
-        return serverSets;
-    }
-
-    protected DLZkServerSet parseServerSet(String serverSetPath) {
-        return DLZkServerSet.of(URI.create(serverSetPath), 60000);
-    }
-
-    @Override
-    public void onStreamsChanged(Iterator<String> streams) {
-        Set<String> newSet = new HashSet<String>();
-        while (streams.hasNext()) {
-            String s = streams.next();
-            if (null == streamRegex || s.matches(streamRegex)) {
-                if (Math.abs(hashFunction.hashUnencodedChars(s).asInt()) % totalInstances == instanceId) {
-                    newSet.add(s);
-                }
-            }
-        }
-        List<StreamChecker> tasksToCancel = new ArrayList<StreamChecker>();
-        synchronized (knownStreams) {
-            Set<String> knownStreamSet = new HashSet<String>(knownStreams.keySet());
-            Set<String> removedStreams = Sets.difference(knownStreamSet, newSet).immutableCopy();
-            Set<String> addedStreams = Sets.difference(newSet, knownStreamSet).immutableCopy();
-            for (String s : removedStreams) {
-                StreamChecker task = knownStreams.remove(s);
-                if (null != task) {
-                    logger.info("Removed stream {}", s);
-                    tasksToCancel.add(task);
-                }
-            }
-            for (String s : addedStreams) {
-                if (!knownStreams.containsKey(s)) {
-                    logger.info("Added stream {}", s);
-                    StreamChecker sc = new StreamChecker(s);
-                    knownStreams.put(s, sc);
-                    sc.run();
-                }
-            }
-        }
-        for (StreamChecker sc : tasksToCancel) {
-            sc.close();
-        }
-    }
-
-    void runMonitor(DistributedLogConfiguration conf, URI dlUri) throws IOException {
-        // stats
-        statsProvider.getStatsLogger("monitor").registerGauge("num_streams", numOfStreamsGauge);
-        logger.info("Construct dl namespace @ {}", dlUri);
-        dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(dlUri)
-                .build();
-        if (watchNamespaceChanges) {
-            dlNamespace.registerNamespaceListener(this);
-        } else {
-            onStreamsChanged(dlNamespace.getLogs());
-        }
-    }
-
-    /**
-     * Close the server.
-     */
-    public void close() {
-        logger.info("Closing monitor service.");
-        if (null != dlClient) {
-            dlClient.close();
-        }
-        if (null != zkServerSets) {
-            for (DLZkServerSet zkServerSet : zkServerSets) {
-                zkServerSet.close();
-            }
-        }
-        if (null != dlNamespace) {
-            dlNamespace.close();
-        }
-        executorService.shutdown();
-        try {
-            if (!executorService.awaitTermination(1, TimeUnit.MINUTES)) {
-                executorService.shutdownNow();
-            }
-        } catch (InterruptedException e) {
-            logger.error("Interrupted on waiting shutting down monitor executor service : ", e);
-        }
-        if (null != statsProvider) {
-            // clean up the gauges
-            unregisterGauge();
-            statsProvider.stop();
-        }
-        keepAliveLatch.countDown();
-        logger.info("Closed monitor service.");
-    }
-
-    public void join() throws InterruptedException {
-        keepAliveLatch.await();
-    }
-
-    /**
-     * clean up the gauge before we close to help GC.
-     */
-    private void unregisterGauge(){
-        statsProvider.getStatsLogger("monitor").unregisterGauge("num_streams", numOfStreamsGauge);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java
deleted file mode 100644
index 1f45b13..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.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 org.apache.distributedlog.service;
-
-import static org.apache.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
-import static org.apache.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
-import static org.apache.distributedlog.util.CommandLineUtils.getOptionalStringArg;
-
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.io.IOException;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-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.cli.ParseException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * The launcher to run monitor service.
- */
-public class MonitorServiceApp {
-
-    private static final Logger logger = LoggerFactory.getLogger(MonitorServiceApp.class);
-
-    static final String USAGE = "MonitorService [-u <uri>] [-c <conf>] [-s serverset]";
-
-    final String[] args;
-    final Options options = new Options();
-
-    private MonitorServiceApp(String[] args) {
-        this.args = args;
-        // prepare options
-        options.addOption("u", "uri", true, "DistributedLog URI");
-        options.addOption("c", "conf", true, "DistributedLog Configuration File");
-        options.addOption("s", "serverset", true, "Proxy Server Set");
-        options.addOption("i", "interval", true, "Check interval");
-        options.addOption("d", "region", true, "Region ID");
-        options.addOption("p", "provider", true, "DistributedLog Stats Provider");
-        options.addOption("f", "filter", true, "Filter streams by regex");
-        options.addOption("w", "watch", false, "Watch stream changes under a given namespace");
-        options.addOption("n", "instance_id", true, "Instance ID");
-        options.addOption("t", "total_instances", true, "Total instances");
-        options.addOption("hck", "heartbeat-num-checks", true, "Send a heartbeat after num checks");
-        options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
-    }
-
-    void printUsage() {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(USAGE, options);
-    }
-
-    private void run() {
-        try {
-            logger.info("Running monitor service.");
-            BasicParser parser = new BasicParser();
-            CommandLine cmdline = parser.parse(options, args);
-            runCmd(cmdline);
-        } catch (ParseException pe) {
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (IOException ie) {
-            logger.error("Failed to start monitor service : ", ie);
-            Runtime.getRuntime().exit(-1);
-        }
-    }
-
-    void runCmd(CommandLine cmdline) throws IOException {
-        StatsProvider statsProvider = new NullStatsProvider();
-        if (cmdline.hasOption("p")) {
-            String providerClass = cmdline.getOptionValue("p");
-            statsProvider = ReflectionUtils.newInstance(providerClass, StatsProvider.class);
-        }
-        StatsReceiver statsReceiver = NullStatsReceiver.get();
-
-        final MonitorService monitorService = new MonitorService(
-                getOptionalStringArg(cmdline, "u"),
-                getOptionalStringArg(cmdline, "c"),
-                getOptionalStringArg(cmdline, "s"),
-                getOptionalIntegerArg(cmdline, "i"),
-                getOptionalIntegerArg(cmdline, "d"),
-                getOptionalStringArg(cmdline, "f"),
-                getOptionalIntegerArg(cmdline, "n"),
-                getOptionalIntegerArg(cmdline, "t"),
-                getOptionalIntegerArg(cmdline, "hck"),
-                getOptionalBooleanArg(cmdline, "hsci"),
-                getOptionalBooleanArg(cmdline, "w"),
-                getOptionalBooleanArg(cmdline, "mx"),
-                statsReceiver,
-                statsProvider);
-
-        monitorService.runServer();
-
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                logger.info("Closing monitor service.");
-                monitorService.close();
-                logger.info("Closed monitor service.");
-            }
-        });
-        try {
-            monitorService.join();
-        } catch (InterruptedException ie) {
-            logger.warn("Interrupted when waiting monitor service to be finished : ", ie);
-        }
-    }
-
-    public static void main(String[] args) {
-        final MonitorServiceApp launcher = new MonitorServiceApp(args);
-        launcher.run();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ResponseUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
deleted file mode 100644
index a2691d3..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.exceptions.DLException;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.thrift.service.BulkWriteResponse;
-import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.thrift.service.StatusCode;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-
-/**
- * Utility methods for building write proxy service responses.
- */
-public class ResponseUtils {
-    public static ResponseHeader deniedHeader() {
-        return new ResponseHeader(StatusCode.REQUEST_DENIED);
-    }
-
-    public static ResponseHeader streamUnavailableHeader() {
-        return new ResponseHeader(StatusCode.STREAM_UNAVAILABLE);
-    }
-
-    public static ResponseHeader successHeader() {
-        return new ResponseHeader(StatusCode.SUCCESS);
-    }
-
-    public static ResponseHeader ownerToHeader(String owner) {
-        return new ResponseHeader(StatusCode.FOUND).setLocation(owner);
-    }
-
-    public static ResponseHeader exceptionToHeader(Throwable t) {
-        ResponseHeader response = new ResponseHeader();
-        if (t instanceof DLException) {
-            DLException dle = (DLException) t;
-            if (dle instanceof OwnershipAcquireFailedException) {
-                response.setLocation(((OwnershipAcquireFailedException) dle).getCurrentOwner());
-            }
-            response.setCode(dle.getCode());
-            response.setErrMsg(dle.getMessage());
-        } else {
-            response.setCode(StatusCode.INTERNAL_SERVER_ERROR);
-            response.setErrMsg("Internal server error : " + t.getMessage());
-        }
-        return response;
-    }
-
-    public static WriteResponse write(ResponseHeader responseHeader) {
-        return new WriteResponse(responseHeader);
-    }
-
-    public static WriteResponse writeSuccess() {
-        return new WriteResponse(successHeader());
-    }
-
-    public static WriteResponse writeDenied() {
-        return new WriteResponse(deniedHeader());
-    }
-
-    public static BulkWriteResponse bulkWrite(ResponseHeader responseHeader) {
-        return new BulkWriteResponse(responseHeader);
-    }
-
-    public static BulkWriteResponse bulkWriteSuccess() {
-        return new BulkWriteResponse(successHeader());
-    }
-
-    public static BulkWriteResponse bulkWriteDenied() {
-        return new BulkWriteResponse(deniedHeader());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java
deleted file mode 100644
index 436145d..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.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 org.apache.distributedlog.service;
-
-/**
- * List of feature keys used by distributedlog server.
- */
-public enum ServerFeatureKeys {
-
-    REGION_STOP_ACCEPT_NEW_STREAM,
-    SERVICE_RATE_LIMIT_DISABLED,
-    SERVICE_CHECKSUM_DISABLED,
-    SERVICE_GLOBAL_LIMITER_DISABLED
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/StatsFilter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/StatsFilter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/StatsFilter.java
deleted file mode 100644
index ee64580..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/StatsFilter.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 org.apache.distributedlog.service;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.SimpleFilter;
-import com.twitter.util.Future;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Track distributedlog server finagle-service stats.
- */
-class StatsFilter<Req, Rep> extends SimpleFilter<Req, Rep> {
-
-    private final StatsLogger stats;
-    private final Counter outstandingAsync;
-    private final OpStatsLogger serviceExec;
-
-    @Override
-    public Future<Rep> apply(Req req, Service<Req, Rep> service) {
-        Future<Rep> result = null;
-        outstandingAsync.inc();
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        try {
-            result = service.apply(req);
-            serviceExec.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-        } finally {
-            outstandingAsync.dec();
-            if (null == result) {
-                serviceExec.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-        }
-        return result;
-    }
-
-    public StatsFilter(StatsLogger stats) {
-        this.stats = stats;
-        this.outstandingAsync = stats.getCounter("outstandingAsync");
-        this.serviceExec = stats.getOpStatsLogger("serviceExec");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java
deleted file mode 100644
index ee64fc7..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/Announcer.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 org.apache.distributedlog.service.announcer;
-
-import java.io.IOException;
-
-/**
- * Announce service information.
- */
-public interface Announcer {
-
-    /**
-     * Announce service info.
-     */
-    void announce() throws IOException;
-
-    /**
-     * Unannounce the service info.
-     */
-    void unannounce() throws IOException;
-
-    /**
-     * Close the announcer.
-     */
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java
deleted file mode 100644
index 5a1277a..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.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 org.apache.distributedlog.service.announcer;
-
-import java.io.IOException;
-
-/**
- * A no-op implementation of {@link Announcer}.
- */
-public class NOPAnnouncer implements Announcer {
-    @Override
-    public void announce() throws IOException {
-        // nop
-    }
-
-    @Override
-    public void unannounce() throws IOException {
-        // nop
-    }
-
-    @Override
-    public void close() {
-        // nop
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
deleted file mode 100644
index df4a8e2..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.announcer;
-
-import com.twitter.common.zookeeper.Group;
-import com.twitter.common.zookeeper.ServerSet;
-import org.apache.distributedlog.client.serverset.DLZkServerSet;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.util.HashMap;
-import java.util.Map;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * ServerSet based announcer.
- */
-public class ServerSetAnnouncer implements Announcer {
-
-    private static final Logger logger = LoggerFactory.getLogger(ServerSetAnnouncer.class);
-
-    final String localAddr;
-    final InetSocketAddress serviceEndpoint;
-    final Map<String, InetSocketAddress> additionalEndpoints;
-    final int shardId;
-
-    // ServerSet
-    DLZkServerSet zkServerSet;
-
-    // Service Status
-    ServerSet.EndpointStatus serviceStatus = null;
-
-    /**
-     * Announce server infos.
-     *
-     * @param servicePort
-     *          service port
-     * @param statsPort
-     *          stats port
-     * @param shardId
-     *          shard id
-     */
-    public ServerSetAnnouncer(URI uri,
-                              int servicePort,
-                              int statsPort,
-                              int shardId) throws UnknownHostException {
-        this.shardId = shardId;
-        this.localAddr = InetAddress.getLocalHost().getHostAddress();
-        // service endpoint
-        this.serviceEndpoint = new InetSocketAddress(localAddr, servicePort);
-        // stats endpoint
-        InetSocketAddress statsEndpoint = new InetSocketAddress(localAddr, statsPort);
-        this.additionalEndpoints = new HashMap<String, InetSocketAddress>();
-        this.additionalEndpoints.put("aurora", statsEndpoint);
-        this.additionalEndpoints.put("stats", statsEndpoint);
-        this.additionalEndpoints.put("service", serviceEndpoint);
-        this.additionalEndpoints.put("thrift", serviceEndpoint);
-
-        // Create zookeeper and server set
-        this.zkServerSet = DLZkServerSet.of(uri, 60000);
-    }
-
-    @Override
-    public synchronized void announce() throws IOException {
-        try {
-            serviceStatus =
-                    zkServerSet.getServerSet().join(serviceEndpoint, additionalEndpoints, shardId);
-        } catch (Group.JoinException e) {
-            throw new IOException("Failed to announce service : ", e);
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted on announcing service : ", e);
-            Thread.currentThread().interrupt();
-        }
-    }
-
-    @Override
-    public synchronized void unannounce() throws IOException {
-        if (null == serviceStatus) {
-            logger.warn("No service to unannounce.");
-            return;
-        }
-        try {
-            serviceStatus.leave();
-        } catch (ServerSet.UpdateException e) {
-            throw new IOException("Failed to unannounce service : ", e);
-        }
-    }
-
-    @Override
-    public void close() {
-        zkServerSet.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/package-info.java
deleted file mode 100644
index 6559bb3..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/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.
- */
-/**
- * Announcers to announce servers to server set.
- */
-package org.apache.distributedlog.service.announcer;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java
deleted file mode 100644
index cdffaa3..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/Balancer.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 org.apache.distributedlog.service.balancer;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-
-/**
- * Balancer Interface.
- *
- * <p>A balancer is used for balance the streams across the proxy cluster.
- */
-public interface Balancer {
-
-    /**
-     * Rebalance all the streams from <i>source</i> to others.
-     *
-     * @param source
-     *          source target name.
-     * @param rebalanceConcurrency
-     *          the concurrency to move streams for re-balance.
-     * @param rebalanceRateLimiter
-     *          the rate limiting to move streams for re-balance.
-     */
-    void balanceAll(String source,
-                    int rebalanceConcurrency,
-                    Optional<RateLimiter> rebalanceRateLimiter);
-
-    /**
-     * Balance the streams across all targets.
-     *
-     * @param rebalanceWaterMark
-     *          rebalance water mark. if number of streams of a given target is less than
-     *          the water mark, no streams will be re-balanced from this target.
-     * @param rebalanceTolerancePercentage
-     *          tolerance percentage for the balancer. if number of streams of a given target is
-     *          less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
-     *          be re-balanced from that target.
-     * @param rebalanceConcurrency
-     *          the concurrency to move streams for re-balance.
-     * @param rebalanceRateLimiter
-     *          the rate limiting to move streams for re-balance.
-     */
-    void balance(int rebalanceWaterMark,
-                 double rebalanceTolerancePercentage,
-                 int rebalanceConcurrency,
-                 Optional<RateLimiter> rebalanceRateLimiter);
-
-    /**
-     * Close the balancer.
-     */
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
deleted file mode 100644
index 964c1cc..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
+++ /dev/null
@@ -1,327 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.common.zookeeper.ServerSet;
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.distributedlog.client.serverset.DLZkServerSet;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.service.ClientUtils;
-import org.apache.distributedlog.service.DLSocketAddress;
-import org.apache.distributedlog.service.DistributedLogClient;
-import org.apache.distributedlog.service.DistributedLogClientBuilder;
-import org.apache.distributedlog.tools.Tool;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Tool to rebalance cluster.
- */
-public class BalancerTool extends Tool {
-
-    private static final Logger logger = LoggerFactory.getLogger(BalancerTool.class);
-
-    static DistributedLogClientBuilder createDistributedLogClientBuilder(ServerSet serverSet) {
-        return DistributedLogClientBuilder.newBuilder()
-                        .name("rebalancer_tool")
-                        .clientId(ClientId$.MODULE$.apply("rebalancer_tool"))
-                        .maxRedirects(2)
-                        .serverSet(serverSet)
-                        .clientBuilder(ClientBuilder.get()
-                                .connectionTimeout(Duration.fromSeconds(2))
-                                .tcpConnectTimeout(Duration.fromSeconds(2))
-                                .requestTimeout(Duration.fromSeconds(10))
-                                .hostConnectionLimit(1)
-                                .hostConnectionCoresize(1)
-                                .keepAlive(true)
-                                .failFast(false));
-    }
-
-    /**
-     * Base Command to run balancer.
-     */
-    protected abstract static class BalancerCommand extends OptsCommand {
-
-        protected Options options = new Options();
-        protected int rebalanceWaterMark = 0;
-        protected double rebalanceTolerancePercentage = 0.0f;
-        protected int rebalanceConcurrency = 1;
-        protected Double rate = null;
-        protected Optional<RateLimiter> rateLimiter;
-
-        BalancerCommand(String name, String description) {
-            super(name, description);
-            options.addOption("rwm", "rebalance-water-mark", true, "Rebalance water mark per proxy");
-            options.addOption("rtp", "rebalance-tolerance-percentage", true,
-                "Rebalance tolerance percentage per proxy");
-            options.addOption("rc", "rebalance-concurrency", true, "Concurrency to rebalance stream distribution");
-            options.addOption("r", "rate", true, "Rebalance rate");
-        }
-
-        Optional<RateLimiter> getRateLimiter() {
-            return rateLimiter;
-        }
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (cmdline.hasOption("rwm")) {
-                this.rebalanceWaterMark = Integer.parseInt(cmdline.getOptionValue("rwm"));
-            }
-            if (cmdline.hasOption("rtp")) {
-                this.rebalanceTolerancePercentage = Double.parseDouble(cmdline.getOptionValue("rtp"));
-            }
-            if (cmdline.hasOption("rc")) {
-                this.rebalanceConcurrency = Integer.parseInt(cmdline.getOptionValue("rc"));
-            }
-            if (cmdline.hasOption("r")) {
-                this.rate = Double.parseDouble(cmdline.getOptionValue("r"));
-            }
-            checkArgument(rebalanceWaterMark >= 0,
-                    "Rebalance Water Mark should be a non-negative number");
-            checkArgument(rebalanceTolerancePercentage >= 0.0f,
-                    "Rebalance Tolerance Percentage should be a non-negative number");
-            checkArgument(rebalanceConcurrency > 0,
-                    "Rebalance Concurrency should be a positive number");
-            if (null == rate || rate <= 0.0f) {
-                rateLimiter = Optional.absent();
-            } else {
-                rateLimiter = Optional.of(RateLimiter.create(rate));
-            }
-        }
-
-        @Override
-        protected int runCmd(CommandLine cmdline) throws Exception {
-            try {
-                parseCommandLine(cmdline);
-            } catch (ParseException pe) {
-                println("ERROR: fail to parse commandline : '" + pe.getMessage() + "'");
-                printUsage();
-                return -1;
-            }
-            return executeCommand(cmdline);
-        }
-
-        protected abstract int executeCommand(CommandLine cmdline) throws Exception;
-    }
-
-    /**
-     * Command to balance streams within a cluster.
-     */
-    protected static class ClusterBalancerCommand extends BalancerCommand {
-
-        protected URI uri;
-        protected String source = null;
-
-        protected ClusterBalancerCommand() {
-            super("clusterbalancer", "Balance streams inside a cluster");
-            options.addOption("u", "uri", true, "DistributedLog URI");
-            options.addOption("sp", "source-proxy", true, "Source proxy to balance");
-        }
-
-        @Override
-        protected String getUsage() {
-            return "clusterbalancer [options]";
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("u")) {
-                throw new ParseException("No proxy serverset provided.");
-            }
-            uri = URI.create(cmdline.getOptionValue("u"));
-            if (cmdline.hasOption("sp")) {
-                String sourceProxyStr = cmdline.getOptionValue("sp");
-                try {
-                    DLSocketAddress.parseSocketAddress(sourceProxyStr);
-                } catch (IllegalArgumentException iae) {
-                    throw new ParseException("Invalid source proxy " + sourceProxyStr + " : " + iae.getMessage());
-                }
-                this.source = sourceProxyStr;
-            }
-        }
-
-        @Override
-        protected int executeCommand(CommandLine cmdline) throws Exception {
-            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
-            logger.info("Created serverset for {}", uri);
-            try {
-                DistributedLogClientBuilder clientBuilder =
-                        createDistributedLogClientBuilder(serverSet.getServerSet());
-                ClusterBalancer balancer = new ClusterBalancer(clientBuilder);
-                try {
-                    return runBalancer(clientBuilder, balancer);
-                } finally {
-                    balancer.close();
-                }
-            } finally {
-                serverSet.close();
-            }
-        }
-
-        protected int runBalancer(DistributedLogClientBuilder clientBuilder,
-                                  ClusterBalancer balancer)
-                throws Exception {
-            if (null == source) {
-                balancer.balance(
-                    rebalanceWaterMark,
-                    rebalanceTolerancePercentage,
-                    rebalanceConcurrency,
-                    getRateLimiter());
-            } else {
-                balanceFromSource(clientBuilder, balancer, source, getRateLimiter());
-            }
-            return 0;
-        }
-
-        protected void balanceFromSource(DistributedLogClientBuilder clientBuilder,
-                                         ClusterBalancer balancer,
-                                         String source,
-                                         Optional<RateLimiter> rateLimiter)
-                throws Exception {
-            InetSocketAddress sourceAddr = DLSocketAddress.parseSocketAddress(source);
-            DistributedLogClientBuilder sourceClientBuilder =
-                    DistributedLogClientBuilder.newBuilder(clientBuilder)
-                            .host(sourceAddr);
-
-            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
-                    ClientUtils.buildClient(sourceClientBuilder);
-            try {
-                Await.result(clientPair.getRight().setAcceptNewStream(false));
-                logger.info("Disable accepting new stream on proxy {}.", source);
-                balancer.balanceAll(source, rebalanceConcurrency, rateLimiter);
-            } finally {
-                clientPair.getLeft().close();
-            }
-        }
-    }
-
-    /**
-     * Command to balance streams between regions.
-     */
-    protected static class RegionBalancerCommand extends BalancerCommand {
-
-        protected URI region1;
-        protected URI region2;
-        protected String source = null;
-
-        protected RegionBalancerCommand() {
-            super("regionbalancer", "Balance streams between regions");
-            options.addOption("rs", "regions", true, "DistributedLog Region URI: uri1[,uri2]");
-            options.addOption("s", "source", true, "DistributedLog Source Region to balance");
-        }
-
-        @Override
-        protected String getUsage() {
-            return "regionbalancer [options]";
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("rs")) {
-                throw new ParseException("No regions provided.");
-            }
-            String regionsStr = cmdline.getOptionValue("rs");
-            String[] regions = regionsStr.split(",");
-            if (regions.length != 2) {
-                throw new ParseException("Invalid regions provided. Expected : serverset1[,serverset2]");
-            }
-            region1 = URI.create(regions[0]);
-            region2 = URI.create(regions[1]);
-            if (cmdline.hasOption("s")) {
-                source = cmdline.getOptionValue("s");
-            }
-        }
-
-        @Override
-        protected int executeCommand(CommandLine cmdline) throws Exception {
-            DLZkServerSet serverSet1 = DLZkServerSet.of(region1, 60000);
-            logger.info("Created serverset for {}", region1);
-            DLZkServerSet serverSet2 = DLZkServerSet.of(region2, 60000);
-            logger.info("Created serverset for {}", region2);
-            try {
-                DistributedLogClientBuilder builder1 =
-                        createDistributedLogClientBuilder(serverSet1.getServerSet());
-                Pair<DistributedLogClient, MonitorServiceClient> pair1 =
-                        ClientUtils.buildClient(builder1);
-                DistributedLogClientBuilder builder2 =
-                        createDistributedLogClientBuilder(serverSet2.getServerSet());
-                Pair<DistributedLogClient, MonitorServiceClient> pair2 =
-                        ClientUtils.buildClient(builder2);
-                try {
-                    SimpleBalancer balancer = new SimpleBalancer(
-                            BKNamespaceDriver.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
-                            BKNamespaceDriver.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
-                    try {
-                        return runBalancer(balancer);
-                    } finally {
-                        balancer.close();
-                    }
-                } finally {
-                    pair1.getLeft().close();
-                    pair2.getLeft().close();
-                }
-            } finally {
-                serverSet1.close();
-                serverSet2.close();
-            }
-        }
-
-        protected int runBalancer(SimpleBalancer balancer) throws Exception {
-            if (null == source) {
-                balancer.balance(
-                    rebalanceWaterMark,
-                    rebalanceTolerancePercentage,
-                    rebalanceConcurrency,
-                    getRateLimiter());
-            } else {
-                balancer.balanceAll(source, rebalanceConcurrency, getRateLimiter());
-            }
-            return 0;
-        }
-    }
-
-    public BalancerTool() {
-        super();
-        addCommand(new ClusterBalancerCommand());
-        addCommand(new RegionBalancerCommand());
-    }
-
-    @Override
-    protected String getName() {
-        return "balancer";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java
deleted file mode 100644
index 4c9e075..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.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 org.apache.distributedlog.service.balancer;
-
-import java.util.Map;
-
-/**
- * Utils for balancer.
- */
-public class BalancerUtils {
-
-    /**
-     * Util function to calculate how many streams to balance for <i>nodeToRebalance</i>,
-     * based on the load distribution <i>loadDistribution</i>.
-     *
-     * @param nodeToRebalance
-     *          node to rebalance
-     * @param loadDistribution
-     *          load distribution map
-     * @param rebalanceWaterMark
-     *          if number of streams of <i>nodeToRebalance</i>
-     *          is less than <i>rebalanceWaterMark</i>, no streams will be re-balanced.
-     * @param tolerancePercentage
-     *          tolerance percentage for the balancer. if number of streams of <i>nodeToRebalance</i>
-     *          is less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
-     *          be re-balanced.
-     * @param <K>
-     * @return number of streams to rebalance
-     */
-    public static <K> int calculateNumStreamsToRebalance(K nodeToRebalance,
-                                                         Map<K, Integer> loadDistribution,
-                                                         int rebalanceWaterMark,
-                                                         double tolerancePercentage) {
-        Integer myLoad = loadDistribution.get(nodeToRebalance);
-        if (null == myLoad || myLoad <= rebalanceWaterMark) {
-            return 0;
-        }
-
-        long totalLoad = 0L;
-        int numNodes = loadDistribution.size();
-
-        for (Map.Entry<K, Integer> entry : loadDistribution.entrySet()) {
-            if (null == entry.getKey() || null == entry.getValue()) {
-                continue;
-            }
-            totalLoad += entry.getValue();
-        }
-
-        double averageLoad = ((double) totalLoad) / numNodes;
-        long permissibleLoad =
-                Math.max(1L, (long) Math.ceil(averageLoad + averageLoad * tolerancePercentage / 100.0f));
-
-        if (myLoad <= permissibleLoad) {
-            return 0;
-        }
-
-        return Math.max(0, myLoad - (int) Math.ceil(averageLoad));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
deleted file mode 100644
index 5add339..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
+++ /dev/null
@@ -1,378 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.distributedlog.service.ClientUtils;
-import org.apache.distributedlog.service.DLSocketAddress;
-import org.apache.distributedlog.service.DistributedLogClient;
-import org.apache.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.util.Await;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.io.Serializable;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A balancer balances ownerships with a cluster of targets.
- */
-public class ClusterBalancer implements Balancer {
-
-    private static final Logger logger = LoggerFactory.getLogger(ClusterBalancer.class);
-
-    /**
-     * Represent a single host. Ordered by number of streams in desc order.
-     */
-    static class Host {
-
-        final SocketAddress address;
-        final Set<String> streams;
-        final DistributedLogClientBuilder clientBuilder;
-        DistributedLogClient client = null;
-        MonitorServiceClient monitor = null;
-
-        Host(SocketAddress address, Set<String> streams,
-             DistributedLogClientBuilder clientBuilder) {
-            this.address = address;
-            this.streams = streams;
-            this.clientBuilder = clientBuilder;
-        }
-
-        private void initializeClientsIfNeeded() {
-            if (null == client) {
-                Pair<DistributedLogClient, MonitorServiceClient> clientPair =
-                        createDistributedLogClient(address, clientBuilder);
-                client = clientPair.getLeft();
-                monitor = clientPair.getRight();
-            }
-        }
-
-        synchronized DistributedLogClient getClient() {
-            initializeClientsIfNeeded();
-            return client;
-        }
-
-        synchronized MonitorServiceClient getMonitor() {
-            initializeClientsIfNeeded();
-            return monitor;
-        }
-
-        synchronized void close() {
-            if (null != client) {
-                client.close();
-            }
-        }
-
-        @Override
-        public String toString() {
-            StringBuilder sb = new StringBuilder();
-            sb.append("Host(").append(address).append(")");
-            return sb.toString();
-        }
-    }
-
-    static class HostComparator implements Comparator<Host>, Serializable {
-        private static final long serialVersionUID = 7984973796525102538L;
-
-        @Override
-        public int compare(Host h1, Host h2) {
-            return h2.streams.size() - h1.streams.size();
-        }
-    }
-
-    protected final DistributedLogClientBuilder clientBuilder;
-    protected final DistributedLogClient client;
-    protected final MonitorServiceClient monitor;
-
-    public ClusterBalancer(DistributedLogClientBuilder clientBuilder) {
-        this(clientBuilder, ClientUtils.buildClient(clientBuilder));
-    }
-
-    ClusterBalancer(DistributedLogClientBuilder clientBuilder,
-                    Pair<DistributedLogClient, MonitorServiceClient> clientPair) {
-        this.clientBuilder = clientBuilder;
-        this.client = clientPair.getLeft();
-        this.monitor = clientPair.getRight();
-    }
-
-    /**
-     * Build a new distributedlog client to a single host <i>host</i>.
-     *
-     * @param host
-     *          host to access
-     * @return distributedlog clients
-     */
-    static Pair<DistributedLogClient, MonitorServiceClient> createDistributedLogClient(
-            SocketAddress host, DistributedLogClientBuilder clientBuilder) {
-        DistributedLogClientBuilder newBuilder =
-                DistributedLogClientBuilder.newBuilder(clientBuilder).host(host);
-        return ClientUtils.buildClient(newBuilder);
-    }
-
-    @Override
-    public void balanceAll(String source,
-                           int rebalanceConcurrency, /* unused */
-                           Optional<RateLimiter> rebalanceRateLimiter) {
-        balance(0, 0.0f, rebalanceConcurrency, Optional.of(source), rebalanceRateLimiter);
-    }
-
-    @Override
-    public void balance(int rebalanceWaterMark,
-                        double rebalanceTolerancePercentage,
-                        int rebalanceConcurrency, /* unused */
-                        Optional<RateLimiter> rebalanceRateLimiter) {
-        Optional<String> source = Optional.absent();
-        balance(rebalanceWaterMark, rebalanceTolerancePercentage, rebalanceConcurrency, source, rebalanceRateLimiter);
-    }
-
-    public void balance(int rebalanceWaterMark,
-                        double rebalanceTolerancePercentage,
-                        int rebalanceConcurrency,
-                        Optional<String> source,
-                        Optional<RateLimiter> rebalanceRateLimiter) {
-        Map<SocketAddress, Set<String>> distribution = monitor.getStreamOwnershipDistribution();
-        if (distribution.size() <= 1) {
-            return;
-        }
-        SocketAddress sourceAddr = null;
-        if (source.isPresent()) {
-            sourceAddr = DLSocketAddress.parseSocketAddress(source.get());
-            logger.info("Balancer source is {}", sourceAddr);
-            if (!distribution.containsKey(sourceAddr)) {
-                return;
-            }
-        }
-        // Get the list of hosts ordered by number of streams in DESC order
-        List<Host> hosts = new ArrayList<Host>(distribution.size());
-        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
-            Host host = new Host(entry.getKey(), entry.getValue(), clientBuilder);
-            hosts.add(host);
-        }
-        Collections.sort(hosts, new HostComparator());
-        try {
-
-            // find the host to move streams from.
-            int hostIdxMoveFrom = -1;
-            if (null != sourceAddr) {
-                for (Host host : hosts) {
-                    ++hostIdxMoveFrom;
-                    if (sourceAddr.equals(host.address)) {
-                        break;
-                    }
-                }
-            }
-
-            // compute the average load.
-            int totalStream = 0;
-            for (Host host : hosts) {
-                totalStream += host.streams.size();
-            }
-            double averageLoad;
-            if (hostIdxMoveFrom >= 0) {
-                averageLoad = ((double) totalStream / (hosts.size() - 1));
-            } else {
-                averageLoad = ((double) totalStream / hosts.size());
-            }
-
-            int moveFromLowWaterMark;
-            int moveToHighWaterMark =
-                Math.max(1, (int) (averageLoad + averageLoad * rebalanceTolerancePercentage / 100.0f));
-
-            if (hostIdxMoveFrom >= 0) {
-                moveFromLowWaterMark = Math.max(0, rebalanceWaterMark);
-                moveStreams(
-                        hosts,
-                        new AtomicInteger(hostIdxMoveFrom), moveFromLowWaterMark,
-                        new AtomicInteger(hosts.size() - 1), moveToHighWaterMark,
-                        rebalanceRateLimiter);
-                moveRemainingStreamsFromSource(hosts.get(hostIdxMoveFrom), hosts, rebalanceRateLimiter);
-            } else {
-                moveFromLowWaterMark = Math.max((int) Math.ceil(averageLoad), rebalanceWaterMark);
-                AtomicInteger moveFrom = new AtomicInteger(0);
-                AtomicInteger moveTo = new AtomicInteger(hosts.size() - 1);
-                while (moveFrom.get() < moveTo.get()) {
-                    moveStreams(hosts, moveFrom, moveFromLowWaterMark,
-                        moveTo, moveToHighWaterMark, rebalanceRateLimiter);
-                    moveFrom.incrementAndGet();
-                }
-            }
-        } finally {
-            for (Host host : hosts) {
-                host.close();
-            }
-        }
-    }
-
-    void moveStreams(List<Host> hosts,
-                     AtomicInteger hostIdxMoveFrom,
-                     int moveFromLowWaterMark,
-                     AtomicInteger hostIdxMoveTo,
-                     int moveToHighWaterMark,
-                     Optional<RateLimiter> rateLimiter) {
-        if (hostIdxMoveFrom.get() < 0 || hostIdxMoveFrom.get() >= hosts.size()
-                || hostIdxMoveTo.get() < 0 || hostIdxMoveTo.get() >= hosts.size()
-                || hostIdxMoveFrom.get() >= hostIdxMoveTo.get()) {
-            return;
-        }
-
-        if (logger.isDebugEnabled()) {
-            logger.debug("Moving streams : hosts = {}, from = {}, to = {} :"
-                + " from_low_water_mark = {}, to_high_water_mark = {}",
-                new Object[] {
-                    hosts,
-                    hostIdxMoveFrom.get(),
-                    hostIdxMoveTo.get(),
-                    moveFromLowWaterMark,
-                    moveToHighWaterMark });
-        }
-
-        Host hostMoveFrom = hosts.get(hostIdxMoveFrom.get());
-        int numStreamsOnFromHost = hostMoveFrom.streams.size();
-        if (numStreamsOnFromHost <= moveFromLowWaterMark) {
-            // do nothing
-            return;
-        }
-
-        int numStreamsToMove = numStreamsOnFromHost - moveFromLowWaterMark;
-        LinkedList<String> streamsToMove = new LinkedList<String>(hostMoveFrom.streams);
-        Collections.shuffle(streamsToMove);
-
-        if (logger.isDebugEnabled()) {
-            logger.debug("Try to move {} streams from host {} : streams = {}",
-                         new Object[] { numStreamsToMove, hostMoveFrom.address, streamsToMove });
-        }
-
-        while (numStreamsToMove-- > 0 && !streamsToMove.isEmpty()) {
-            if (rateLimiter.isPresent()) {
-                rateLimiter.get().acquire();
-            }
-
-            // pick a host to move
-            Host hostMoveTo = hosts.get(hostIdxMoveTo.get());
-            while (hostMoveTo.streams.size() >= moveToHighWaterMark) {
-                int hostIdx = hostIdxMoveTo.decrementAndGet();
-                logger.info("move to host : {}, from {}", hostIdx, hostIdxMoveFrom.get());
-                if (hostIdx <= hostIdxMoveFrom.get()) {
-                    return;
-                } else {
-                    hostMoveTo = hosts.get(hostIdx);
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("Target host to move moved to host {} @ {}",
-                                hostIdx, hostMoveTo);
-                    }
-                }
-            }
-
-            // pick a stream
-            String stream = streamsToMove.remove();
-
-            // move the stream
-            if (moveStream(stream, hostMoveFrom, hostMoveTo)) {
-                hostMoveFrom.streams.remove(stream);
-                hostMoveTo.streams.add(stream);
-            }
-        }
-
-    }
-
-    void moveRemainingStreamsFromSource(Host source,
-                                        List<Host> hosts,
-                                        Optional<RateLimiter> rateLimiter) {
-        LinkedList<String> streamsToMove = new LinkedList<String>(source.streams);
-        Collections.shuffle(streamsToMove);
-
-        if (logger.isDebugEnabled()) {
-            logger.debug("Try to move remaining streams from {} : {}", source, streamsToMove);
-        }
-
-        int hostIdx = hosts.size() - 1;
-
-        while (!streamsToMove.isEmpty()) {
-            if (rateLimiter.isPresent()) {
-                rateLimiter.get().acquire();
-            }
-
-            Host target = hosts.get(hostIdx);
-            if (!target.address.equals(source.address)) {
-                String stream = streamsToMove.remove();
-                // move the stream
-                if (moveStream(stream, source, target)) {
-                    source.streams.remove(stream);
-                    target.streams.add(stream);
-                }
-            }
-            --hostIdx;
-            if (hostIdx < 0) {
-                hostIdx = hosts.size() - 1;
-            }
-        }
-    }
-
-    private boolean moveStream(String stream, Host from, Host to) {
-        try {
-            doMoveStream(stream, from, to);
-            return true;
-        } catch (Exception e) {
-            return false;
-        }
-    }
-
-    private void doMoveStream(final String stream, final Host from, final Host to) throws Exception {
-        logger.info("Moving stream {} from {} to {}.",
-                    new Object[] { stream, from.address, to.address });
-        Await.result(from.getClient().release(stream).flatMap(new Function<Void, Future<Void>>() {
-            @Override
-            public Future<Void> apply(Void result) {
-                logger.info("Released stream {} from {}.", stream, from.address);
-                return to.getMonitor().check(stream).addEventListener(new FutureEventListener<Void>() {
-
-                    @Override
-                    public void onSuccess(Void value) {
-                        logger.info("Moved stream {} from {} to {}.",
-                                    new Object[] { stream, from.address, to.address });
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        logger.info("Failed to move stream {} from {} to {} : ",
-                                    new Object[] { stream, from.address, to.address, cause });
-                    }
-                });
-            }
-        }));
-    }
-
-    @Override
-    public void close() {
-        client.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java
deleted file mode 100644
index 6a43179..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.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 org.apache.distributedlog.service.balancer;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.Serializable;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * A stream chooser based on number of streams.
- */
-class CountBasedStreamChooser implements StreamChooser, Serializable,
-        Comparator<Pair<SocketAddress, LinkedList<String>>> {
-
-    private static final long serialVersionUID = 4664153397369979203L;
-
-    final List<Pair<SocketAddress, LinkedList<String>>> streamsDistribution;
-
-    // pivot index in the list of hosts. the chooser will remove streams from the hosts before
-    // pivot, which will reduce their stream counts to make them equal to the stream count of the pivot.
-    int pivot;
-    int pivotCount;
-
-    // next index in the list of hosts to choose stream from.
-    int next;
-
-    CountBasedStreamChooser(Map<SocketAddress, Set<String>> streams) {
-        checkArgument(streams.size() > 0, "Only support no-empty streams distribution");
-        streamsDistribution = new ArrayList<Pair<SocketAddress, LinkedList<String>>>(streams.size());
-        for (Map.Entry<SocketAddress, Set<String>> entry : streams.entrySet()) {
-            LinkedList<String> randomizedStreams = new LinkedList<String>(entry.getValue());
-            Collections.shuffle(randomizedStreams);
-            streamsDistribution.add(Pair.of(entry.getKey(), randomizedStreams));
-        }
-        // sort the hosts by the number of streams in descending order
-        Collections.sort(streamsDistribution, this);
-        pivot = 0;
-        pivotCount = streamsDistribution.get(0).getValue().size();
-        findNextPivot();
-        next = 0;
-    }
-
-    private void findNextPivot() {
-        int prevPivotCount = pivotCount;
-        while (++pivot < streamsDistribution.size()) {
-            pivotCount = streamsDistribution.get(pivot).getValue().size();
-            if (pivotCount < prevPivotCount) {
-                return;
-            }
-        }
-        pivot = streamsDistribution.size();
-        pivotCount = 0;
-    }
-
-    @Override
-    public synchronized String choose() {
-        // reach the pivot
-        if (next == pivot) {
-            if (streamsDistribution.get(next - 1).getRight().size() > pivotCount) {
-                next = 0;
-            } else if (pivotCount == 0) { // the streams are empty now
-                return null;
-            } else {
-                findNextPivot();
-                next = 0;
-            }
-        }
-
-        // get stream count that next host to choose from
-        LinkedList<String> nextStreams = streamsDistribution.get(next).getRight();
-        if (nextStreams.size() == 0) {
-            return null;
-        }
-
-        String chosenStream = nextStreams.remove();
-        ++next;
-        return chosenStream;
-    }
-
-    @Override
-    public int compare(Pair<SocketAddress, LinkedList<String>> o1,
-                       Pair<SocketAddress, LinkedList<String>> o2) {
-        return o2.getValue().size() - o1.getValue().size();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java
deleted file mode 100644
index 4aefc5e..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.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 org.apache.distributedlog.service.balancer;
-
-/**
- * A stream chooser that can only choose limited number of streams.
- */
-public class LimitedStreamChooser implements StreamChooser {
-
-  /**
-   * Create a limited stream chooser by {@code limit}.
-   *
-   * @param underlying the underlying stream chooser.
-   * @param limit the limit of number of streams to choose.
-   * @return the limited stream chooser.
-   */
-    public static LimitedStreamChooser of(StreamChooser underlying, int limit) {
-        return new LimitedStreamChooser(underlying, limit);
-    }
-
-    final StreamChooser underlying;
-    int limit;
-
-    LimitedStreamChooser(StreamChooser underlying, int limit) {
-        this.underlying = underlying;
-        this.limit = limit;
-    }
-
-    @Override
-    public synchronized String choose() {
-        if (limit <= 0) {
-            return null;
-        }
-        String s = underlying.choose();
-        if (s == null) {
-            limit = 0;
-            return null;
-        }
-        --limit;
-        return s;
-    }
-}



[29/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 1300187..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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 });
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index b3f3368..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index ed6269b..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index d7e2c94..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.monitor;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index f3c24ca..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 486bd6f..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.ownership;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index d22b0da..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 9b5c7f6..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 769cca8..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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();
-
-}



[16/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
new file mode 100644
index 0000000..3c53ccf
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
@@ -0,0 +1,246 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.DistributedLogClient;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A balancer balances ownerships between two targets.
+ */
+public class SimpleBalancer implements Balancer {
+
+    private static final Logger logger = LoggerFactory.getLogger(SimpleBalancer.class);
+
+    protected final String target1;
+    protected final String target2;
+    protected final DistributedLogClient targetClient1;
+    protected final DistributedLogClient targetClient2;
+    protected final MonitorServiceClient targetMonitor1;
+    protected final MonitorServiceClient targetMonitor2;
+
+    public SimpleBalancer(String name1,
+                          DistributedLogClient client1,
+                          MonitorServiceClient monitor1,
+                          String name2,
+                          DistributedLogClient client2,
+                          MonitorServiceClient monitor2) {
+        this.target1 = name1;
+        this.targetClient1 = client1;
+        this.targetMonitor1 = monitor1;
+        this.target2 = name2;
+        this.targetClient2 = client2;
+        this.targetMonitor2 = monitor2;
+    }
+
+    protected static int countNumberStreams(Map<SocketAddress, Set<String>> distribution) {
+        int count = 0;
+        for (Set<String> streams : distribution.values()) {
+            count += streams.size();
+        }
+        return count;
+    }
+
+    @Override
+    public void balance(int rebalanceWaterMark,
+                        double rebalanceTolerancePercentage,
+                        int rebalanceConcurrency,
+                        Optional<RateLimiter> rebalanceRateLimiter) {
+        // get the ownership distributions from individual targets
+        Map<SocketAddress, Set<String>> distribution1 = targetMonitor1.getStreamOwnershipDistribution();
+        Map<SocketAddress, Set<String>> distribution2 = targetMonitor2.getStreamOwnershipDistribution();
+
+        // get stream counts
+        int proxyCount1 = distribution1.size();
+        int streamCount1 = countNumberStreams(distribution1);
+        int proxyCount2 = distribution2.size();
+        int streamCount2 = countNumberStreams(distribution2);
+
+        logger.info("'{}' has {} streams by {} proxies; while '{}' has {} streams by {} proxies.",
+                    new Object[] {target1, streamCount1, proxyCount1, target2, streamCount2, proxyCount2 });
+
+        String source, target;
+        Map<SocketAddress, Set<String>> srcDistribution;
+        DistributedLogClient srcClient, targetClient;
+        MonitorServiceClient srcMonitor, targetMonitor;
+        int srcStreamCount, targetStreamCount;
+        if (streamCount1 > streamCount2) {
+            source = target1;
+            srcStreamCount = streamCount1;
+            srcClient = targetClient1;
+            srcMonitor = targetMonitor1;
+            srcDistribution = distribution1;
+
+            target = target2;
+            targetStreamCount = streamCount2;
+            targetClient = targetClient2;
+            targetMonitor = targetMonitor2;
+        } else {
+            source = target2;
+            srcStreamCount = streamCount2;
+            srcClient = targetClient2;
+            srcMonitor = targetMonitor2;
+            srcDistribution = distribution2;
+
+            target = target1;
+            targetStreamCount = streamCount1;
+            targetClient = targetClient1;
+            targetMonitor = targetMonitor1;
+        }
+
+        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
+        loadDistribution.put(source, srcStreamCount);
+        loadDistribution.put(target, targetStreamCount);
+
+        // Calculate how many streams to be rebalanced from src region to target region
+        int numStreamsToRebalance = BalancerUtils.calculateNumStreamsToRebalance(
+            source, loadDistribution, rebalanceWaterMark, rebalanceTolerancePercentage);
+
+        if (numStreamsToRebalance <= 0) {
+            logger.info("No streams need to be rebalanced from '{}' to '{}'.", source, target);
+            return;
+        }
+
+        StreamChooser streamChooser =
+                LimitedStreamChooser.of(new CountBasedStreamChooser(srcDistribution), numStreamsToRebalance);
+        StreamMover streamMover =
+            new StreamMoverImpl(source, srcClient, srcMonitor, target, targetClient, targetMonitor);
+
+        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
+    }
+
+    @Override
+    public void balanceAll(String source,
+                           int rebalanceConcurrency,
+                           Optional<RateLimiter> rebalanceRateLimiter) {
+        String target;
+        DistributedLogClient sourceClient, targetClient;
+        MonitorServiceClient sourceMonitor, targetMonitor;
+        if (target1.equals(source)) {
+            sourceClient = targetClient1;
+            sourceMonitor = targetMonitor1;
+            target = target2;
+            targetClient = targetClient2;
+            targetMonitor = targetMonitor2;
+        } else if (target2.equals(source)) {
+            sourceClient = targetClient2;
+            sourceMonitor = targetMonitor2;
+            target = target1;
+            targetClient = targetClient1;
+            targetMonitor = targetMonitor1;
+        } else {
+            throw new IllegalArgumentException("Unknown target " + source);
+        }
+
+        // get the ownership distributions from individual targets
+        Map<SocketAddress, Set<String>> distribution = sourceMonitor.getStreamOwnershipDistribution();
+
+        if (distribution.isEmpty()) {
+            return;
+        }
+
+        StreamChooser streamChooser = new CountBasedStreamChooser(distribution);
+        StreamMover streamMover =
+            new StreamMoverImpl(source, sourceClient, sourceMonitor, target, targetClient, targetMonitor);
+
+        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
+    }
+
+    private void moveStreams(StreamChooser streamChooser,
+                             StreamMover streamMover,
+                             int concurrency,
+                             Optional<RateLimiter> rateLimiter) {
+        CountDownLatch doneLatch = new CountDownLatch(concurrency);
+        RegionMover regionMover = new RegionMover(streamChooser, streamMover, rateLimiter, doneLatch);
+        ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
+        try {
+            for (int i = 0; i < concurrency; i++) {
+                executorService.submit(regionMover);
+            }
+
+            try {
+                doneLatch.await();
+            } catch (InterruptedException e) {
+                logger.info("{} is interrupted. Stopping it ...", streamMover);
+                regionMover.shutdown();
+            }
+        } finally {
+            executorService.shutdown();
+        }
+
+    }
+
+    /**
+     * Move streams from <i>src</i> region to <i>target</i> region.
+     */
+    static class RegionMover implements Runnable {
+
+        final StreamChooser streamChooser;
+        final StreamMover streamMover;
+        final Optional<RateLimiter> rateLimiter;
+        final CountDownLatch doneLatch;
+        volatile boolean running = true;
+
+        RegionMover(StreamChooser streamChooser,
+                    StreamMover streamMover,
+                    Optional<RateLimiter> rateLimiter,
+                    CountDownLatch doneLatch) {
+            this.streamChooser = streamChooser;
+            this.streamMover = streamMover;
+            this.rateLimiter = rateLimiter;
+            this.doneLatch = doneLatch;
+        }
+
+        @Override
+        public void run() {
+            while (running) {
+                if (rateLimiter.isPresent()) {
+                    rateLimiter.get().acquire();
+                }
+
+                String stream = streamChooser.choose();
+                if (null == stream) {
+                    break;
+                }
+
+                streamMover.moveStream(stream);
+            }
+            doneLatch.countDown();
+        }
+
+        void shutdown() {
+            running = false;
+        }
+    }
+
+    @Override
+    public void close() {
+        // no-op
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
new file mode 100644
index 0000000..1d7b6f7
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+/**
+ * Choose a stream to rebalance.
+ */
+public interface StreamChooser {
+    /**
+     * Choose a stream to rebalance.
+     *
+     * @return stream chose
+     */
+    String choose();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java
new file mode 100644
index 0000000..4a04530
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.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.service.balancer;
+
+/**
+ * A stream mover to move streams between proxies.
+ */
+public interface StreamMover {
+
+    /**
+     * Move given stream <i>streamName</i>.
+     *
+     * @param streamName
+     *          stream name to move
+     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
+     * @throws Exception
+     */
+    boolean moveStream(final String streamName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
new file mode 100644
index 0000000..68d934b
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.util.Await;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Move Streams from <i>src</i> to <i>target</i>.
+ */
+public class StreamMoverImpl implements StreamMover {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamMoverImpl.class);
+
+    final String source, target;
+    final DistributedLogClient srcClient, targetClient;
+    final MonitorServiceClient srcMonitor, targetMonitor;
+
+    public StreamMoverImpl(String source, DistributedLogClient srcClient, MonitorServiceClient srcMonitor,
+                           String target, DistributedLogClient targetClient, MonitorServiceClient targetMonitor) {
+        this.source = source;
+        this.srcClient = srcClient;
+        this.srcMonitor = srcMonitor;
+        this.target = target;
+        this.targetClient = targetClient;
+        this.targetMonitor = targetMonitor;
+    }
+
+    /**
+     * Move given stream <i>streamName</i>.
+     *
+     * @param streamName
+     *          stream name to move
+     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
+     * @throws Exception
+     */
+    public boolean moveStream(final String streamName) {
+        try {
+            doMoveStream(streamName);
+            return true;
+        } catch (Exception e) {
+            return false;
+        }
+    }
+
+    private void doMoveStream(final String streamName) throws Exception {
+        Await.result(srcClient.release(streamName).flatMap(new Function<Void, Future<Void>>() {
+            @Override
+            public Future<Void> apply(Void result) {
+                return targetMonitor.check(streamName).addEventListener(new FutureEventListener<Void>() {
+                    @Override
+                    public void onSuccess(Void value) {
+                        logger.info("Moved stream {} from {} to {}.",
+                                new Object[]{streamName, source, target});
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        logger.info("Failed to move stream {} from region {} to {} : ",
+                                new Object[]{streamName, source, target, cause});
+                    }
+                });
+            }
+        }));
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("StreamMover('").append(source).append("' -> '").append(target).append("')");
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/package-info.java
new file mode 100644
index 0000000..9eb8950
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/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.
+ */
+/**
+ * Balancer to move streams around to balance the traffic.
+ */
+package org.apache.distributedlog.service.balancer;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
new file mode 100644
index 0000000..7d72093
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.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.service.config;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.config.ConfigurationSubscription;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.config.FileConfigurationBuilder;
+import org.apache.distributedlog.config.PropertiesConfigurationBuilder;
+import java.io.File;
+import java.net.MalformedURLException;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.configuration.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * For all streams return the same dynamic config based on configFile.
+ */
+public class DefaultStreamConfigProvider implements StreamConfigProvider {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultStreamConfigProvider.class);
+
+    private final Optional<DynamicDistributedLogConfiguration> dynConf;
+    private final ConfigurationSubscription confSub;
+
+    public DefaultStreamConfigProvider(String configFilePath,
+                                       ScheduledExecutorService executorService,
+                                       int reloadPeriod,
+                                       TimeUnit reloadUnit)
+        throws ConfigurationException {
+        try {
+            File configFile = new File(configFilePath);
+            FileConfigurationBuilder properties =
+                new PropertiesConfigurationBuilder(configFile.toURI().toURL());
+            ConcurrentConstConfiguration defaultConf =
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+            DynamicDistributedLogConfiguration conf =
+                new DynamicDistributedLogConfiguration(defaultConf);
+            List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(properties);
+            confSub = new ConfigurationSubscription(
+                conf, fileConfigBuilders, executorService, reloadPeriod, reloadUnit);
+            this.dynConf = Optional.of(conf);
+        } catch (MalformedURLException ex) {
+            throw new ConfigurationException(ex);
+        }
+    }
+
+    @Override
+    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
+        return dynConf;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java
new file mode 100644
index 0000000..195f29d
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.config;
+
+import com.google.common.base.Optional;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * For all streams return an absent configuration.
+ */
+public class NullStreamConfigProvider implements StreamConfigProvider {
+    static final Logger LOG = LoggerFactory.getLogger(NullStreamConfigProvider.class);
+
+    private static final Optional<DynamicDistributedLogConfiguration> nullConf =
+            Optional.<DynamicDistributedLogConfiguration>absent();
+
+    @Override
+    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
+        return nullConf;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
new file mode 100644
index 0000000..257b4be
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
@@ -0,0 +1,443 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.config;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.SystemConfiguration;
+
+/**
+ * Configuration for DistributedLog Server.
+ */
+public class ServerConfiguration extends CompositeConfiguration {
+
+    private static ClassLoader defaultLoader;
+
+    static {
+        defaultLoader = Thread.currentThread().getContextClassLoader();
+        if (null == defaultLoader) {
+            defaultLoader = DistributedLogConfiguration.class.getClassLoader();
+        }
+    }
+
+    // Server DLSN version
+    protected static final String SERVER_DLSN_VERSION = "server_dlsn_version";
+    protected static final byte SERVER_DLSN_VERSION_DEFAULT = DLSN.VERSION1;
+
+    // Server Durable Write Enable/Disable Flag
+    protected static final String SERVER_DURABLE_WRITE_ENABLED = "server_durable_write_enabled";
+    protected static final boolean SERVER_DURABLE_WRITE_ENABLED_DEFAULT = true;
+
+    // Server Region Id
+    protected static final String SERVER_REGION_ID = "server_region_id";
+    protected static final int SERVER_REGION_ID_DEFAULT = DistributedLogConstants.LOCAL_REGION_ID;
+
+    // Server Port
+    protected static final String SERVER_PORT = "server_port";
+    protected static final int SERVER_PORT_DEFAULT = 0;
+
+    // Server Shard Id
+    protected static final String SERVER_SHARD_ID = "server_shard";
+    protected static final int SERVER_SHARD_ID_DEFAULT = -1;
+
+    // Server Threads
+    protected static final String SERVER_NUM_THREADS = "server_threads";
+    protected static final int SERVER_NUM_THREADS_DEFAULT = Runtime.getRuntime().availableProcessors();
+
+    // Server enable per stream stat
+    protected static final String SERVER_ENABLE_PERSTREAM_STAT = "server_enable_perstream_stat";
+    protected static final boolean SERVER_ENABLE_PERSTREAM_STAT_DEFAULT = true;
+
+    // Server graceful shutdown period (in millis)
+    protected static final String SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS = "server_graceful_shutdown_period_ms";
+    protected static final long SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT = 0L;
+
+    // Server service timeout
+    public static final String SERVER_SERVICE_TIMEOUT_MS = "server_service_timeout_ms";
+    public static final String SERVER_SERVICE_TIMEOUT_MS_OLD = "serviceTimeoutMs";
+    public static final long SERVER_SERVICE_TIMEOUT_MS_DEFAULT = 0;
+
+    // Server close writer timeout
+    public static final String SERVER_WRITER_CLOSE_TIMEOUT_MS = "server_writer_close_timeout_ms";
+    public static final long SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT = 1000;
+
+    // Server stream probation timeout
+    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS = "server_stream_probation_timeout_ms";
+    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD = "streamProbationTimeoutMs";
+    public static final long SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT = 60 * 1000 * 5;
+
+    // Server stream to partition converter
+    protected static final String SERVER_STREAM_PARTITION_CONVERTER_CLASS = "stream_partition_converter_class";
+
+    // Use hostname as the allocator pool name
+    protected static final String SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME =
+        "server_use_hostname_as_allocator_pool_name";
+    protected static final boolean SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT = false;
+    //Configure refresh interval for calculating resource placement in seconds
+    public static final String SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S =
+        "server_resource_placement_refresh_interval_sec";
+    public static final int  SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT = 120;
+
+    public ServerConfiguration() {
+        super();
+        addConfiguration(new SystemConfiguration());
+    }
+
+    /**
+     * Load configurations from {@link DistributedLogConfiguration}.
+     *
+     * @param dlConf
+     *          distributedlog configuration
+     */
+    public void loadConf(DistributedLogConfiguration dlConf) {
+        addConfiguration(dlConf);
+    }
+
+    /**
+     * Set the version to encode dlsn.
+     *
+     * @param version
+     *          dlsn version
+     * @return server configuration
+     */
+    public ServerConfiguration setDlsnVersion(byte version) {
+        setProperty(SERVER_DLSN_VERSION, version);
+        return this;
+    }
+
+    /**
+     * Get the version to encode dlsn.
+     *
+     * @see DLSN
+     * @return version to encode dlsn.
+     */
+    public byte getDlsnVersion() {
+        return getByte(SERVER_DLSN_VERSION, SERVER_DLSN_VERSION_DEFAULT);
+    }
+
+    /**
+     * Set the flag to enable/disable durable write.
+     *
+     * @param enabled
+     *          flag to enable/disable durable write
+     * @return server configuration
+     */
+    public ServerConfiguration enableDurableWrite(boolean enabled) {
+        setProperty(SERVER_DURABLE_WRITE_ENABLED, enabled);
+        return this;
+    }
+
+    /**
+     * Is durable write enabled.
+     *
+     * @return true if waiting writes to be durable. otherwise false.
+     */
+    public boolean isDurableWriteEnabled() {
+        return getBoolean(SERVER_DURABLE_WRITE_ENABLED, SERVER_DURABLE_WRITE_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Set the region id used to instantiate DistributedLogNamespace.
+     *
+     * @param regionId
+     *          region id
+     * @return server configuration
+     */
+    public ServerConfiguration setRegionId(int regionId) {
+        setProperty(SERVER_REGION_ID, regionId);
+        return this;
+    }
+
+    /**
+     * Get the region id used to instantiate {@link org.apache.distributedlog.namespace.DistributedLogNamespace}.
+     *
+     * @return region id used to instantiate DistributedLogNamespace
+     */
+    public int getRegionId() {
+        return getInt(SERVER_REGION_ID, SERVER_REGION_ID_DEFAULT);
+    }
+
+    /**
+     * Set the server port running for this service.
+     *
+     * @param port
+     *          server port
+     * @return server configuration
+     */
+    public ServerConfiguration setServerPort(int port) {
+        setProperty(SERVER_PORT, port);
+        return this;
+    }
+
+    /**
+     * Get the server port running for this service.
+     *
+     * @return server port
+     */
+    public int getServerPort() {
+        return getInt(SERVER_PORT, SERVER_PORT_DEFAULT);
+    }
+
+    /**
+     * Set the shard id of this server.
+     *
+     * @param shardId
+     *          shard id
+     * @return shard id of this server
+     */
+    public ServerConfiguration setServerShardId(int shardId) {
+        setProperty(SERVER_SHARD_ID, shardId);
+        return this;
+    }
+
+    /**
+     * Get the shard id of this server.
+     *
+     * <p>It would be used to instantiate the client id used for DistributedLogNamespace.
+     *
+     * @return shard id of this server.
+     */
+    public int getServerShardId() {
+        return getInt(SERVER_SHARD_ID, SERVER_SHARD_ID_DEFAULT);
+    }
+
+    /**
+     * Get the number of threads for the executor of this server.
+     *
+     * @return number of threads for the executor running in this server.
+     */
+    public int getServerThreads() {
+        return getInt(SERVER_NUM_THREADS, SERVER_NUM_THREADS_DEFAULT);
+    }
+
+    /**
+     * Set the number of threads for the executor of this server.
+     *
+     * @param numThreads
+     *          number of threads for the executor running in this server.
+     * @return server configuration
+     */
+    public ServerConfiguration setServerThreads(int numThreads) {
+        setProperty(SERVER_NUM_THREADS, numThreads);
+        return this;
+    }
+
+    /**
+     * Enable/Disable per stream stat.
+     *
+     * @param enabled
+     *          flag to enable/disable per stream stat
+     * @return server configuration
+     */
+    public ServerConfiguration setPerStreamStatEnabled(boolean enabled) {
+        setProperty(SERVER_ENABLE_PERSTREAM_STAT, enabled);
+        return this;
+    }
+
+    /**
+     * Whether the per stream stat enabled for not in this server.
+     *
+     * @return true if per stream stat enable, otherwise false.
+     */
+    public boolean isPerStreamStatEnabled() {
+        return getBoolean(SERVER_ENABLE_PERSTREAM_STAT, SERVER_ENABLE_PERSTREAM_STAT_DEFAULT);
+    }
+
+    /**
+     * Set the graceful shutdown period in millis.
+     *
+     * @param periodMs
+     *          graceful shutdown period in millis.
+     * @return server configuration
+     */
+    public ServerConfiguration setGracefulShutdownPeriodMs(long periodMs) {
+        setProperty(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, periodMs);
+        return this;
+    }
+
+    /**
+     * Get the graceful shutdown period in millis.
+     *
+     * @return graceful shutdown period in millis.
+     */
+    public long getGracefulShutdownPeriodMs() {
+        return getLong(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT);
+    }
+
+    /**
+     * Get timeout for stream op execution in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @return timeout for stream operation in proxy layer.
+     */
+    public long getServiceTimeoutMs() {
+        return getLong(SERVER_SERVICE_TIMEOUT_MS,
+                getLong(SERVER_SERVICE_TIMEOUT_MS_OLD, SERVER_SERVICE_TIMEOUT_MS_DEFAULT));
+    }
+
+    /**
+     * Set timeout for stream op execution in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @param timeoutMs
+     *          timeout for stream operation in proxy layer.
+     * @return dl configuration.
+     */
+    public ServerConfiguration setServiceTimeoutMs(long timeoutMs) {
+        setProperty(SERVER_SERVICE_TIMEOUT_MS, timeoutMs);
+        return this;
+    }
+
+    /**
+     * Get timeout for closing writer in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @return timeout for closing writer in proxy layer.
+     */
+    public long getWriterCloseTimeoutMs() {
+        return getLong(SERVER_WRITER_CLOSE_TIMEOUT_MS, SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT);
+    }
+
+    /**
+     * Set timeout for closing writer in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @param timeoutMs
+     *          timeout for closing writer in proxy layer.
+     * @return dl configuration.
+     */
+    public ServerConfiguration setWriterCloseTimeoutMs(long timeoutMs) {
+        setProperty(SERVER_WRITER_CLOSE_TIMEOUT_MS, timeoutMs);
+        return this;
+    }
+
+    /**
+     * How long should stream be kept in cache in probationary state after service timeout.
+     *
+     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
+     *
+     * @return stream probation timeout in ms.
+     */
+    public long getStreamProbationTimeoutMs() {
+        return getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS,
+                getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD, SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT));
+    }
+
+    /**
+     * How long should stream be kept in cache in probationary state after service timeout.
+     *
+     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
+     *
+     * @param timeoutMs probation timeout in ms.
+     * @return server configuration
+     */
+    public ServerConfiguration setStreamProbationTimeoutMs(long timeoutMs) {
+        setProperty(SERVER_STREAM_PROBATION_TIMEOUT_MS, timeoutMs);
+        return this;
+    }
+
+    /**
+     * Set the stream partition converter class.
+     *
+     * @param converterClass
+     *          stream partition converter class
+     * @return server configuration
+     */
+    public ServerConfiguration setStreamPartitionConverterClass(
+        Class<? extends StreamPartitionConverter> converterClass) {
+        setProperty(SERVER_STREAM_PARTITION_CONVERTER_CLASS, converterClass.getName());
+        return this;
+    }
+
+    /**
+     * Get the stream partition converter class.
+     *
+     * @return the stream partition converter class.
+     * @throws ConfigurationException
+     */
+    public Class<? extends StreamPartitionConverter> getStreamPartitionConverterClass()
+            throws ConfigurationException {
+        return ReflectionUtils.getClass(
+                this,
+                SERVER_STREAM_PARTITION_CONVERTER_CLASS,
+                IdentityStreamPartitionConverter.class,
+                StreamPartitionConverter.class,
+                defaultLoader);
+    }
+
+     /**
+      * Set if use hostname as the allocator pool name.
+      *
+      * @param useHostname whether to use hostname as the allocator pool name.
+      * @return server configuration
+      * @see #isUseHostnameAsAllocatorPoolName()
+      */
+    public ServerConfiguration setUseHostnameAsAllocatorPoolName(boolean useHostname) {
+        setProperty(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME, useHostname);
+        return this;
+    }
+
+    /**
+     * Get if use hostname as the allocator pool name.
+     *
+     * @return true if use hostname as the allocator pool name. otherwise, use
+     * {@link #getServerShardId()} as the allocator pool name.
+     * @see #getServerShardId()
+     */
+    public boolean isUseHostnameAsAllocatorPoolName() {
+        return getBoolean(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME,
+            SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT);
+    }
+
+    public ServerConfiguration setResourcePlacementRefreshInterval(int refreshIntervalSecs) {
+        setProperty(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, refreshIntervalSecs);
+        return this;
+    }
+
+    public int getResourcePlacementRefreshInterval() {
+        return getInt(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT);
+    }
+
+    /**
+     * Validate the configuration.
+     *
+     * @throws IllegalStateException when there are any invalid settings.
+     */
+    public void validate() {
+        byte dlsnVersion = getDlsnVersion();
+        checkArgument(dlsnVersion >= DLSN.VERSION0 && dlsnVersion <= DLSN.VERSION1,
+                "Unknown dlsn version " + dlsnVersion);
+        checkArgument(getServerThreads() > 0,
+                "Invalid number of server threads : " + getServerThreads());
+        checkArgument(getServerShardId() >= 0,
+                "Invalid server shard id : " + getServerShardId());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java
new file mode 100644
index 0000000..29052f9
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.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.service.config;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.config.DynamicConfigurationFactory;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import java.io.File;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.configuration.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provide per stream configuration to DistributedLog service layer.
+ */
+public class ServiceStreamConfigProvider implements StreamConfigProvider {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ServiceStreamConfigProvider.class);
+
+    private static final String CONFIG_EXTENSION = "conf";
+
+    private final File configBaseDir;
+    private final File defaultConfigFile;
+    private final StreamPartitionConverter partitionConverter;
+    private final DynamicConfigurationFactory configFactory;
+    private final DynamicDistributedLogConfiguration defaultDynConf;
+
+    public ServiceStreamConfigProvider(String configPath,
+                                       String defaultConfigPath,
+                                       StreamPartitionConverter partitionConverter,
+                                       ScheduledExecutorService executorService,
+                                       int reloadPeriod,
+                                       TimeUnit reloadUnit)
+                                       throws ConfigurationException {
+        this.configBaseDir = new File(configPath);
+        if (!configBaseDir.exists()) {
+            throw new ConfigurationException("Stream configuration base directory "
+                + configPath + " does not exist");
+        }
+        this.defaultConfigFile = new File(configPath);
+        if (!defaultConfigFile.exists()) {
+            throw new ConfigurationException("Stream configuration default config "
+                + defaultConfigPath + " does not exist");
+        }
+
+        // Construct reloading default configuration
+        this.partitionConverter = partitionConverter;
+        this.configFactory = new DynamicConfigurationFactory(executorService, reloadPeriod, reloadUnit);
+        // We know it exists from the check above.
+        this.defaultDynConf = configFactory.getDynamicConfiguration(defaultConfigPath).get();
+    }
+
+    @Override
+    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
+        String configName = partitionConverter.convert(streamName).getStream();
+        String configPath = getConfigPath(configName);
+        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.<DynamicDistributedLogConfiguration>absent();
+        try {
+            dynConf = configFactory.getDynamicConfiguration(configPath, defaultDynConf);
+        } catch (ConfigurationException ex) {
+            LOG.warn("Configuration exception for stream {} ({}) at {}",
+                    new Object[] {streamName, configName, configPath, ex});
+        }
+        return dynConf;
+    }
+
+    private String getConfigPath(String configName) {
+        return new File(configBaseDir, String.format("%s.%s", configName, CONFIG_EXTENSION)).getPath();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java
new file mode 100644
index 0000000..c704f70
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.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.service.config;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
+/**
+ * Expose per-stream configs to dl proxy.
+ */
+public interface StreamConfigProvider {
+    /**
+     * Get dynamic per stream config overrides for a given stream.
+     *
+     * @param streamName stream name to return config for
+     * @return Optional dynamic configuration instance
+     */
+    Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/package-info.java
new file mode 100644
index 0000000..b07605e
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/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 Server Configurations.
+ */
+package org.apache.distributedlog.service.config;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/package-info.java
new file mode 100644
index 0000000..3fcfeda
--- /dev/null
+++ b/distributedlog-proxy-server/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 Proxy Service.
+ */
+package org.apache.distributedlog.service;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java
new file mode 100644
index 0000000..fa3dd49
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import com.twitter.util.Future;
+
+/**
+ * Equal Load Appraiser.
+ *
+ * <p>Created for those who hold these truths to be self-evident, that all streams are created equal,
+ * that they are endowed by their creator with certain unalienable loads, that among these are
+ * Uno, Eins, and One.
+ */
+public class EqualLoadAppraiser implements LoadAppraiser {
+    @Override
+    public Future<StreamLoad> getStreamLoad(String stream) {
+        return Future.value(new StreamLoad(stream, 1));
+    }
+
+    @Override
+    public Future<Void> refreshCache() {
+        return Future.value(null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
new file mode 100644
index 0000000..2e9dd6b
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.util.Duration;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.Futures;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Least Load Placement Policy.
+ *
+ * <p>A LoadPlacementPolicy that attempts to place streams in such a way that the load is balanced as
+ * evenly as possible across all shards. The LoadAppraiser remains responsible for determining what
+ * the load of a server would be. This placement policy then distributes these streams across the
+ * servers.
+ */
+public class LeastLoadPlacementPolicy extends PlacementPolicy {
+
+    private static final Logger logger = LoggerFactory.getLogger(LeastLoadPlacementPolicy.class);
+
+    private TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
+    private Map<String, String> streamToServer = new HashMap<String, String>();
+
+    public LeastLoadPlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
+                                    DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
+                                    Duration refreshInterval, StatsLogger statsLogger) {
+        super(loadAppraiser, routingService, namespace, placementStateManager, refreshInterval, statsLogger);
+        statsLogger.registerGauge("placement/load.diff", new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                if (serverLoads.size() > 0) {
+                    return serverLoads.last().getLoad() - serverLoads.first().getLoad();
+                } else {
+                    return getDefaultValue();
+                }
+            }
+        });
+    }
+
+    private synchronized String getStreamOwner(String stream) {
+        return streamToServer.get(stream);
+    }
+
+    @Override
+    public Future<String> placeStream(String stream) {
+        String streamOwner = getStreamOwner(stream);
+        if (null != streamOwner) {
+            return Future.value(streamOwner);
+        }
+        Future<StreamLoad> streamLoadFuture = loadAppraiser.getStreamLoad(stream);
+        return streamLoadFuture.map(new Function<StreamLoad, String>() {
+            @Override
+            public String apply(StreamLoad streamLoad) {
+                return placeStreamSynchronized(streamLoad);
+            }
+        });
+    }
+
+    private synchronized String placeStreamSynchronized(StreamLoad streamLoad) {
+        ServerLoad serverLoad = serverLoads.pollFirst();
+        serverLoad.addStream(streamLoad);
+        serverLoads.add(serverLoad);
+        return serverLoad.getServer();
+    }
+
+    @Override
+    public void refresh() {
+        logger.info("Refreshing server loads.");
+        Future<Void> refresh = loadAppraiser.refreshCache();
+        final Set<String> servers = getServers();
+        final Set<String> allStreams = getStreams();
+        Future<TreeSet<ServerLoad>> serverLoadsFuture = refresh.flatMap(
+            new Function<Void, Future<TreeSet<ServerLoad>>>() {
+            @Override
+            public Future<TreeSet<ServerLoad>> apply(Void v1) {
+                return calculate(servers, allStreams);
+            }
+        });
+        serverLoadsFuture.map(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
+                try {
+                    updateServerLoads(serverLoads);
+                } catch (PlacementStateManager.StateManagerSaveException e) {
+                    logger.error("The refreshed mapping could not be persisted and will not be used.", e);
+                }
+                return BoxedUnit.UNIT;
+            }
+        });
+    }
+
+    private synchronized void updateServerLoads(TreeSet<ServerLoad> serverLoads)
+        throws PlacementStateManager.StateManagerSaveException {
+        this.placementStateManager.saveOwnership(serverLoads);
+        this.streamToServer = serverLoadsToMap(serverLoads);
+        this.serverLoads = serverLoads;
+    }
+
+    @Override
+    public synchronized void load(TreeSet<ServerLoad> serverLoads) {
+        this.serverLoads = serverLoads;
+        this.streamToServer = serverLoadsToMap(serverLoads);
+    }
+
+    public Future<TreeSet<ServerLoad>> calculate(final Set<String> servers, Set<String> streams) {
+        logger.info("Calculating server loads");
+        final long startTime = System.currentTimeMillis();
+        ArrayList<Future<StreamLoad>> futures = new ArrayList<Future<StreamLoad>>(streams.size());
+
+        for (String stream : streams) {
+            Future<StreamLoad> streamLoad = loadAppraiser.getStreamLoad(stream);
+            futures.add(streamLoad);
+        }
+
+        return Futures.collect(futures).map(new Function<List<StreamLoad>, TreeSet<ServerLoad>>() {
+            @Override
+            public TreeSet<ServerLoad> apply(List<StreamLoad> streamLoads) {
+        /* Sort streamLoads so largest streams are placed first for better balance */
+                TreeSet<StreamLoad> streamQueue = new TreeSet<StreamLoad>();
+                for (StreamLoad streamLoad : streamLoads) {
+                    streamQueue.add(streamLoad);
+                }
+
+                TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
+                for (String server : servers) {
+                    ServerLoad serverLoad = new ServerLoad(server);
+                    if (!streamQueue.isEmpty()) {
+                        serverLoad.addStream(streamQueue.pollFirst());
+                    }
+                    serverLoads.add(serverLoad);
+                }
+
+                while (!streamQueue.isEmpty()) {
+                    ServerLoad serverLoad = serverLoads.pollFirst();
+                    serverLoad.addStream(streamQueue.pollFirst());
+                    serverLoads.add(serverLoad);
+                }
+                return serverLoads;
+            }
+        }).onSuccess(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
+                placementCalcStats.registerSuccessfulEvent(System.currentTimeMillis() - startTime);
+                return BoxedUnit.UNIT;
+            }
+        }).onFailure(new Function<Throwable, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(Throwable t) {
+                logger.error("Failure calculating loads", t);
+                placementCalcStats.registerFailedEvent(System.currentTimeMillis() - startTime);
+                return BoxedUnit.UNIT;
+            }
+        });
+    }
+
+    private static Map<String, String> serverLoadsToMap(Collection<ServerLoad> serverLoads) {
+        HashMap<String, String> streamToServer = new HashMap<String, String>(serverLoads.size());
+        for (ServerLoad serverLoad : serverLoads) {
+            for (StreamLoad streamLoad : serverLoad.getStreamLoads()) {
+                streamToServer.put(streamLoad.getStream(), serverLoad.getServer());
+            }
+        }
+        return streamToServer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java
new file mode 100644
index 0000000..5cd8980
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.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.service.placement;
+
+import com.twitter.util.Future;
+
+/**
+ * Interface for load appraiser.
+ */
+public interface LoadAppraiser {
+    /**
+     * Retrieve the stream load for a given {@code stream}.
+     *
+     * @param stream name of the stream
+     * @return the stream load of the stream.
+     */
+    Future<StreamLoad> getStreamLoad(String stream);
+
+    /**
+     * Refesch the cache.
+     * @return
+     */
+    Future<Void> refreshCache();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
new file mode 100644
index 0000000..ac952aa
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.util.Duration;
+import com.twitter.util.Function0;
+import com.twitter.util.Future;
+import com.twitter.util.ScheduledThreadPoolTimer;
+import com.twitter.util.Time;
+import com.twitter.util.Timer;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A PlacementPolicy assigns streams to servers given an appraisal of the load that the stream contains.
+ *
+ * <p>The load of a stream is determined by the LoadAppraiser used. The PlacementPolicy will
+ * then distributed these StreamLoads to the available servers in a manner defined by the
+ * implementation creating ServerLoad objects. It then saves this assignment via the
+ * PlacementStateManager.
+ */
+public abstract class PlacementPolicy {
+
+    private static final Logger logger = LoggerFactory.getLogger(PlacementPolicy.class);
+
+    protected final LoadAppraiser loadAppraiser;
+    protected final RoutingService routingService;
+    protected final DistributedLogNamespace namespace;
+    protected final PlacementStateManager placementStateManager;
+    private final Duration refreshInterval;
+    protected final OpStatsLogger placementCalcStats;
+    private Timer placementRefreshTimer;
+
+    public PlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
+                           DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
+                           Duration refreshInterval, StatsLogger statsLogger) {
+        this.loadAppraiser = loadAppraiser;
+        this.routingService = routingService;
+        this.namespace = namespace;
+        this.placementStateManager = placementStateManager;
+        this.refreshInterval = refreshInterval;
+        placementCalcStats = statsLogger.getOpStatsLogger("placement");
+    }
+
+    public Set<String> getServers() {
+        Set<SocketAddress> hosts = routingService.getHosts();
+        Set<String> servers = new HashSet<String>(hosts.size());
+        for (SocketAddress address : hosts) {
+            servers.add(DLSocketAddress.toString((InetSocketAddress) address));
+        }
+        return servers;
+    }
+
+    public Set<String> getStreams() {
+        Set<String> streams = new HashSet<String>();
+        try {
+            Iterator<String> logs = namespace.getLogs();
+            while (logs.hasNext()) {
+                streams.add(logs.next());
+            }
+        } catch (IOException e) {
+            logger.error("Could not get streams for placement policy.", e);
+        }
+        return streams;
+    }
+
+    public void start(boolean leader) {
+        logger.info("Starting placement policy");
+
+        TreeSet<ServerLoad> emptyServerLoads = new TreeSet<ServerLoad>();
+        for (String server : getServers()) {
+            emptyServerLoads.add(new ServerLoad(server));
+        }
+        load(emptyServerLoads); //Pre-Load so streams don't NPE
+        if (leader) { //this is the leader shard
+            logger.info("Shard is leader. Scheduling timed refresh.");
+            placementRefreshTimer = new ScheduledThreadPoolTimer(1, "timer", true);
+            placementRefreshTimer.schedule(Time.now(), refreshInterval, new Function0<BoxedUnit>() {
+                @Override
+                public BoxedUnit apply() {
+                    refresh();
+                    return BoxedUnit.UNIT;
+                }
+            });
+        } else {
+            logger.info("Shard is not leader. Watching for server load changes.");
+            placementStateManager.watch(new PlacementStateManager.PlacementCallback() {
+                @Override
+                public void callback(TreeSet<ServerLoad> serverLoads) {
+                    if (!serverLoads.isEmpty()) {
+                        load(serverLoads);
+                    }
+                }
+            });
+        }
+    }
+
+    public void close() {
+        if (placementRefreshTimer != null) {
+            placementRefreshTimer.stop();
+        }
+    }
+
+    /**
+     * Places the stream on a server according to the policy.
+     *
+     * <p>It returns a future containing the host that owns the stream upon completion
+     */
+    public abstract Future<String> placeStream(String stream);
+
+    /**
+     * Recalculates the entire placement mapping and updates stores it using the PlacementStateManager.
+     */
+    public abstract void refresh();
+
+    /**
+     * Loads the placement mapping into the node from a TreeSet of ServerLoads.
+     */
+    public abstract void load(TreeSet<ServerLoad> serverLoads);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
new file mode 100644
index 0000000..0187bed
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import java.util.TreeSet;
+
+/**
+ * The PlacementStateManager handles persistence of calculated resource placements.
+ */
+public interface PlacementStateManager {
+
+    /**
+     * Saves the ownership mapping as a TreeSet of ServerLoads to persistent storage.
+     */
+    void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException;
+
+    /**
+     * Loads the ownership mapping as TreeSet of ServerLoads from persistent storage.
+     */
+    TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException;
+
+    /**
+     * Watch the persistent storage for changes to the ownership mapping.
+     *
+     * <p>The placementCallback callbacks will be triggered with the new mapping when a change occurs.
+     */
+    void watch(PlacementCallback placementCallback);
+
+    /**
+     * Placement Callback.
+     *
+     * <p>The callback is triggered when server loads are updated.
+     */
+    interface PlacementCallback {
+        void callback(TreeSet<ServerLoad> serverLoads);
+    }
+
+    /**
+     * The base exception thrown when state manager encounters errors.
+     */
+    abstract class StateManagerException extends Exception {
+        public StateManagerException(String message, Exception e) {
+            super(message, e);
+        }
+    }
+
+    /**
+     * Exception thrown when failed to load the ownership mapping.
+     */
+    class StateManagerLoadException extends StateManagerException {
+        public StateManagerLoadException(Exception e) {
+            super("Load of Ownership failed", e);
+        }
+    }
+
+    /**
+     * Exception thrown when failed to save the ownership mapping.
+     */
+    class StateManagerSaveException extends StateManagerException {
+        public StateManagerSaveException(Exception e) {
+            super("Save of Ownership failed", e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java
new file mode 100644
index 0000000..d65c401
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.apache.thrift.transport.TMemoryInputTransport;
+
+/**
+ * An object represents the server load.
+ *
+ * <p>A comparable data object containing the identifier of the server, total appraised load on the
+ * server, and all streams assigned to the server by the resource placement mapping. This is
+ * comparable first by load and then by server so that a sorted data structure of these will be
+ * consistent across multiple calculations.
+ */
+public class ServerLoad implements Comparable {
+    private static final int BUFFER_SIZE = 4096000;
+    private final String server;
+    private final HashSet<StreamLoad> streamLoads = new HashSet<StreamLoad>();
+    private long load = 0L;
+
+    public ServerLoad(String server) {
+        this.server = server;
+    }
+
+    public synchronized long addStream(StreamLoad stream) {
+        this.load += stream.getLoad();
+        streamLoads.add(stream);
+        return this.load;
+    }
+
+    public synchronized long removeStream(String stream) {
+        for (StreamLoad streamLoad : streamLoads) {
+            if (streamLoad.stream.equals(stream)) {
+                this.load -= streamLoad.getLoad();
+                streamLoads.remove(streamLoad);
+                return this.load;
+            }
+        }
+        return this.load; //Throwing an exception wouldn't help us as our logic should never reach here
+    }
+
+    public synchronized long getLoad() {
+        return load;
+    }
+
+    public synchronized Set<StreamLoad> getStreamLoads() {
+        return streamLoads;
+    }
+
+    public synchronized String getServer() {
+        return server;
+    }
+
+    protected synchronized org.apache.distributedlog.service.placement.thrift.ServerLoad toThrift() {
+        org.apache.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
+            new org.apache.distributedlog.service.placement.thrift.ServerLoad();
+        tServerLoad.setServer(server);
+        tServerLoad.setLoad(load);
+        ArrayList<org.apache.distributedlog.service.placement.thrift.StreamLoad> tStreamLoads =
+            new ArrayList<org.apache.distributedlog.service.placement.thrift.StreamLoad>();
+        for (StreamLoad streamLoad : streamLoads) {
+            tStreamLoads.add(streamLoad.toThrift());
+        }
+        tServerLoad.setStreams(tStreamLoads);
+        return tServerLoad;
+    }
+
+    public byte[] serialize() throws IOException {
+        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            toThrift().write(protocol);
+            transport.flush();
+            return transport.toString(UTF_8.name()).getBytes(UTF_8);
+        } catch (TException e) {
+            throw new IOException("Failed to serialize server load : ", e);
+        } catch (UnsupportedEncodingException uee) {
+            throw new IOException("Failed to serialize server load : ", uee);
+        }
+    }
+
+    public static ServerLoad deserialize(byte[] data) throws IOException {
+        org.apache.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
+            new org.apache.distributedlog.service.placement.thrift.ServerLoad();
+        TMemoryInputTransport transport = new TMemoryInputTransport(data);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            tServerLoad.read(protocol);
+            ServerLoad serverLoad = new ServerLoad(tServerLoad.getServer());
+            if (tServerLoad.isSetStreams()) {
+                for (org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad :
+                    tServerLoad.getStreams()) {
+                    serverLoad.addStream(new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad()));
+                }
+            }
+            return serverLoad;
+        } catch (TException e) {
+            throw new IOException("Failed to deserialize server load : ", e);
+        }
+    }
+
+    @Override
+    public synchronized int compareTo(Object o) {
+        ServerLoad other = (ServerLoad) o;
+        if (load == other.getLoad()) {
+            return server.compareTo(other.getServer());
+        } else {
+            return Long.compare(load, other.getLoad());
+        }
+    }
+
+    @Override
+    public synchronized boolean equals(Object o) {
+        if (!(o instanceof ServerLoad)) {
+            return false;
+        }
+        ServerLoad other = (ServerLoad) o;
+        return server.equals(other.getServer())
+            && load == other.getLoad()
+            && streamLoads.equals(other.getStreamLoads());
+    }
+
+    @Override
+    public synchronized String toString() {
+        return String.format("ServerLoad<Server: %s, Load: %d, Streams: %s>", server, load, streamLoads);
+    }
+
+    @Override
+    public synchronized int hashCode() {
+        return new HashCodeBuilder().append(server).append(load).append(streamLoads).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java
new file mode 100644
index 0000000..f271222
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.apache.thrift.transport.TMemoryInputTransport;
+
+/**
+ * An object represent the load of a stream.
+ *
+ * <p>A comparable data object containing the identifier of the stream and the appraised load produced
+ * by the stream.
+ */
+public class StreamLoad implements Comparable {
+    private static final int BUFFER_SIZE = 4096;
+    public final String stream;
+    private final int load;
+
+    public StreamLoad(String stream, int load) {
+        this.stream = stream;
+        this.load = load;
+    }
+
+    public int getLoad() {
+        return load;
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    protected org.apache.distributedlog.service.placement.thrift.StreamLoad toThrift() {
+        org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
+            new org.apache.distributedlog.service.placement.thrift.StreamLoad();
+        return tStreamLoad.setStream(stream).setLoad(load);
+    }
+
+    public byte[] serialize() throws IOException {
+        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            toThrift().write(protocol);
+            transport.flush();
+            return transport.toString(UTF_8.name()).getBytes(UTF_8);
+        } catch (TException e) {
+            throw new IOException("Failed to serialize stream load : ", e);
+        } catch (UnsupportedEncodingException uee) {
+            throw new IOException("Failed to serialize stream load : ", uee);
+        }
+    }
+
+    public static StreamLoad deserialize(byte[] data) throws IOException {
+        org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
+            new org.apache.distributedlog.service.placement.thrift.StreamLoad();
+        TMemoryInputTransport transport = new TMemoryInputTransport(data);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            tStreamLoad.read(protocol);
+            return new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad());
+        } catch (TException e) {
+            throw new IOException("Failed to deserialize stream load : ", e);
+        }
+    }
+
+    @Override
+    public int compareTo(Object o) {
+        StreamLoad other = (StreamLoad) o;
+        if (load == other.getLoad()) {
+            return stream.compareTo(other.getStream());
+        } else {
+            return Long.compare(load, other.getLoad());
+        }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof StreamLoad)) {
+            return false;
+        }
+        StreamLoad other = (StreamLoad) o;
+        return stream.equals(other.getStream()) && load == other.getLoad();
+    }
+
+    @Override
+    public String toString() {
+        return String.format("StreamLoad<Stream: %s, Load: %d>", stream, load);
+    }
+
+    @Override
+    public int hashCode() {
+        return new HashCodeBuilder().append(stream).append(load).build();
+    }
+}



[22/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
new file mode 100644
index 0000000..666fa31
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
new file mode 100644
index 0000000..e51eb1e
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
new file mode 100644
index 0000000..d71cee3
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
new file mode 100644
index 0000000..ad73c17
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
new file mode 100644
index 0000000..4ac22ce
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java
new file mode 100644
index 0000000..8e8edd3
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
new file mode 100644
index 0000000..4fe8141
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
new file mode 100644
index 0000000..77b7beb
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
new file mode 100644
index 0000000..753a1af
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
new file mode 100644
index 0000000..2fc8de0
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
new file mode 100644
index 0000000..1ff7c93
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
new file mode 100644
index 0000000..352d755
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java
new file mode 100644
index 0000000..93cdf7a
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
new file mode 100644
index 0000000..38a7544
--- /dev/null
+++ b/distributedlog-proxy-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;


[21/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..f1da33c
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..faf45c2
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
new file mode 100644
index 0000000..68fe8b0
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
new file mode 100644
index 0000000..4bdd4b1
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
new file mode 100644
index 0000000..c2dcddd
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java
new file mode 100644
index 0000000..530c632
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java
new file mode 100644
index 0000000..7a49faa
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
new file mode 100644
index 0000000..b94b4be
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
new file mode 100644
index 0000000..110e99a
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
new file mode 100644
index 0000000..106d3fc
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
new file mode 100644
index 0000000..68e6825
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
new file mode 100644
index 0000000..9f30815
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
new file mode 100644
index 0000000..0e2a152
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/package-info.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/service/package-info.java
new file mode 100644
index 0000000..033882f
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/main/resources/findbugsExclude.xml b/distributedlog-proxy-client/src/main/resources/findbugsExclude.xml
new file mode 100644
index 0000000..05ee085
--- /dev/null
+++ b/distributedlog-proxy-client/src/main/resources/findbugsExclude.xml
@@ -0,0 +1,23 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+//-->
+<FindBugsFilter>
+  <Match>
+    <!-- generated code, we can't be held responsible for findbugs in it //-->
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
+  </Match>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
new file mode 100644
index 0000000..d7494de
--- /dev/null
+++ b/distributedlog-proxy-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();
+    }
+}



[11/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
new file mode 100644
index 0000000..d0a2f88
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.DistributedLogConfiguration;
+import org.apache.distributedlog.feature.DefaultFeatureProvider;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link org.apache.distributedlog.exceptions.RegionUnavailableException}.
+ */
+public class TestRegionUnavailable extends DistributedLogServerTestCase {
+
+    /**
+     * A feature provider for testing.
+     */
+    public static class TestFeatureProvider extends DefaultFeatureProvider {
+
+        public TestFeatureProvider(String rootScope,
+                                   DistributedLogConfiguration conf,
+                                   StatsLogger statsLogger) {
+            super(rootScope, conf, statsLogger);
+        }
+
+        @Override
+        protected Feature makeFeature(String featureName) {
+            if (featureName.contains(ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase())) {
+                return new SettableFeature(featureName, 10000);
+            }
+            return super.makeFeature(featureName);
+        }
+
+        @Override
+        protected FeatureProvider makeProvider(String fullScopeName) {
+            return super.makeProvider(fullScopeName);
+        }
+    }
+
+    private final int numServersPerDC = 3;
+    private final List<DLServer> localCluster;
+    private final List<DLServer> remoteCluster;
+    private TwoRegionDLClient client;
+
+    public TestRegionUnavailable() {
+        super(true);
+        this.localCluster = new ArrayList<DLServer>();
+        this.remoteCluster = new ArrayList<DLServer>();
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        DistributedLogConfiguration localConf = new DistributedLogConfiguration();
+        localConf.addConfiguration(conf);
+        localConf.setFeatureProviderClass(TestFeatureProvider.class);
+        DistributedLogConfiguration remoteConf = new DistributedLogConfiguration();
+        remoteConf.addConfiguration(conf);
+        super.setup();
+        int localPort = 9010;
+        int remotePort = 9020;
+        for (int i = 0; i < numServersPerDC; i++) {
+            localCluster.add(createDistributedLogServer(localConf, localPort + i));
+            remoteCluster.add(createDistributedLogServer(remoteConf, remotePort + i));
+        }
+        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
+        for (DLServer server : localCluster) {
+            regionMap.put(server.getAddress(), "local");
+        }
+        for (DLServer server : remoteCluster) {
+            regionMap.put(server.getAddress(), "remote");
+        }
+        client = createTwoRegionDLClient("two_regions_client", regionMap);
+
+    }
+
+    private void registerStream(String streamName) {
+        for (DLServer server : localCluster) {
+            client.localRoutingService.addHost(streamName, server.getAddress());
+        }
+        client.remoteRoutingService.addHost(streamName, remoteCluster.get(0).getAddress());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != client) {
+            client.shutdown();
+        }
+        for (DLServer server : localCluster) {
+            server.shutdown();
+        }
+        for (DLServer server : remoteCluster) {
+            server.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testRegionUnavailable() throws Exception {
+        String name = "dlserver-region-unavailable";
+        registerStream(name);
+
+        for (long i = 1; i <= 10; i++) {
+            client.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
+        }
+
+        // check local region
+        for (DLServer server : localCluster) {
+            checkStreams(0, server);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java
new file mode 100644
index 0000000..c8b8bdf
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.assertEquals;
+
+import com.twitter.finagle.Service;
+import com.twitter.finagle.service.ConstantService;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StatsFilter}.
+ */
+public class TestStatsFilter {
+
+    class RuntimeExService<Req, Rep> extends Service<Req, Rep> {
+        public Future<Rep> apply(Req request) {
+            throw new RuntimeException("test");
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceSuccess() throws Exception {
+        StatsLogger stats = new NullStatsLogger();
+        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
+        Future<String> result = filter.apply("", new ConstantService<String, String>(Future.value("result")));
+        assertEquals("result", Await.result(result));
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceFailure() throws Exception {
+        StatsLogger stats = new NullStatsLogger();
+        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
+        try {
+            filter.apply("", new RuntimeExService<String, String>());
+        } catch (RuntimeException ex) {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java
new file mode 100644
index 0000000..21bebb5
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link BalancerUtils}.
+ */
+public class TestBalancerUtils {
+
+    @Test(timeout = 60000)
+    public void testCalculateNumStreamsToRebalance() {
+        String myNode = "mynode";
+
+        // empty load distribution
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, new HashMap<String, Integer>(), 0, 10));
+        // my node doesn't exist in load distribution
+        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
+        loadDistribution.put("node1", 10);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 0, 10));
+        // my node doesn't reach rebalance water mark
+        loadDistribution.clear();
+        loadDistribution.put("node1", 1);
+        loadDistribution.put(myNode, 100);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 200, 10));
+        // my node is below average in the cluster.
+        loadDistribution.clear();
+        loadDistribution.put(myNode, 1);
+        loadDistribution.put("node1", 99);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 0, 10));
+        // my node is above average in the cluster
+        assertEquals(49, BalancerUtils.calculateNumStreamsToRebalance(
+                "node1", loadDistribution, 0, 10));
+        // my node is at the tolerance range
+        loadDistribution.clear();
+        loadDistribution.put(myNode, 55);
+        loadDistribution.put("node1", 45);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 0, 10));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
new file mode 100644
index 0000000..fb3fb6e
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.fail;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.DistributedLogServerTestCase;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link ClusterBalancer}.
+ */
+public class TestClusterBalancer extends DistributedLogServerTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestClusterBalancer.class);
+
+    private final int numServers = 5;
+    private final List<DLServer> cluster;
+    private DLClient client;
+
+    public TestClusterBalancer() {
+        super(true);
+        this.cluster = new ArrayList<DLServer>();
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        int initPort = 9001;
+        for (int i = 0; i < numServers; i++) {
+            cluster.add(createDistributedLogServer(initPort + i));
+        }
+        client = createDistributedLogClient("cluster_client", Optional.<String>absent());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != client) {
+            client.shutdown();
+        }
+        for (DLServer server: cluster) {
+            server.shutdown();
+        }
+    }
+
+    private void initStreams(String namePrefix) {
+        logger.info("Init streams with prefix {}", namePrefix);
+        // Stream Distribution: 5, 4, 3, 2, 1
+        initStreams(namePrefix, 5, 1, 0);
+        initStreams(namePrefix, 4, 6, 1);
+        initStreams(namePrefix, 3, 10, 2);
+        initStreams(namePrefix, 2, 13, 3);
+        initStreams(namePrefix, 1, 15, 4);
+    }
+
+    private void initStreams(String namePrefix, int numStreams, int streamId, int proxyId) {
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + (streamId++);
+            client.routingService.addHost(name, cluster.get(proxyId).getAddress());
+        }
+    }
+
+    private void writeStreams(String namePrefix) throws Exception {
+        logger.info("Write streams with prefix {}", namePrefix);
+        writeStreams(namePrefix, 5, 1);
+        writeStreams(namePrefix, 4, 6);
+        writeStreams(namePrefix, 3, 10);
+        writeStreams(namePrefix, 2, 13);
+        writeStreams(namePrefix, 1, 15);
+    }
+
+    private void writeStreams(String namePrefix, int numStreams, int streamId) throws Exception {
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + (streamId++);
+            try {
+                Await.result(client.dlClient.write(name, ByteBuffer.wrap(name.getBytes(UTF_8))));
+            } catch (Exception e) {
+                logger.error("Error writing stream {} : ", name, e);
+                throw e;
+            }
+        }
+    }
+
+    private void validateStreams(String namePrefix) throws Exception {
+        logger.info("Validate streams with prefix {}", namePrefix);
+        validateStreams(namePrefix, 5, 1, 0);
+        validateStreams(namePrefix, 4, 6, 1);
+        validateStreams(namePrefix, 3, 10, 2);
+        validateStreams(namePrefix, 2, 13, 3);
+        validateStreams(namePrefix, 1, 15, 4);
+    }
+
+    private void validateStreams(String namePrefix, int numStreams, int streamId, int proxyIdx) {
+        Set<String> expectedStreams = new HashSet<String>();
+        for (int i = 0; i < numStreams; i++) {
+            expectedStreams.add(namePrefix + (streamId++));
+        }
+        checkStreams(expectedStreams, cluster.get(proxyIdx));
+    }
+
+    @Ignore
+    @Test(timeout = 60000)
+    public void testBalanceAll() throws Exception {
+        String namePrefix = "clusterbalancer-balance-all-";
+
+        initStreams(namePrefix);
+        writeStreams(namePrefix);
+        validateStreams(namePrefix);
+
+        Optional<RateLimiter> rateLimiter = Optional.absent();
+
+        Balancer balancer = new ClusterBalancer(client.dlClientBuilder,
+                Pair.of((DistributedLogClient) client.dlClient, (MonitorServiceClient) client.dlClient));
+        logger.info("Rebalancing from 'unknown' target");
+        try {
+            balancer.balanceAll("unknown", 10, rateLimiter);
+            fail("Should fail on balanceAll from 'unknown' target.");
+        } catch (IllegalArgumentException iae) {
+            // expected
+        }
+        validateStreams(namePrefix);
+
+        logger.info("Rebalancing from 'unexisted' host");
+        String addr = DLSocketAddress.toString(DLSocketAddress.getSocketAddress(9999));
+        balancer.balanceAll(addr, 10, rateLimiter);
+        validateStreams(namePrefix);
+
+        addr = DLSocketAddress.toString(cluster.get(0).getAddress());
+        logger.info("Rebalancing from host {}.", addr);
+        balancer.balanceAll(addr, 10, rateLimiter);
+        checkStreams(0, cluster.get(0));
+        checkStreams(4, cluster.get(1));
+        checkStreams(3, cluster.get(2));
+        checkStreams(4, cluster.get(3));
+        checkStreams(4, cluster.get(4));
+
+        addr = DLSocketAddress.toString(cluster.get(2).getAddress());
+        logger.info("Rebalancing from host {}.", addr);
+        balancer.balanceAll(addr, 10, rateLimiter);
+        checkStreams(3, cluster.get(0));
+        checkStreams(4, cluster.get(1));
+        checkStreams(0, cluster.get(2));
+        checkStreams(4, cluster.get(3));
+        checkStreams(4, cluster.get(4));
+
+        logger.info("Rebalancing the cluster");
+        balancer.balance(0, 0.0f, 10, rateLimiter);
+        for (int i = 0; i < 5; i++) {
+            checkStreams(3, cluster.get(i));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
new file mode 100644
index 0000000..6734083
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Sets;
+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 org.junit.Test;
+
+/**
+ * Test Case for {@link CountBasedStreamChooser}.
+ */
+public class TestCountBasedStreamChooser {
+
+    @Test(timeout = 60000)
+    public void testEmptyStreamDistribution() {
+        try {
+            new CountBasedStreamChooser(new HashMap<SocketAddress, Set<String>>());
+            fail("Should fail constructing stream chooser if the stream distribution is empty");
+        } catch (IllegalArgumentException iae) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testMultipleHostsWithEmptyStreams() {
+        for (int i = 1; i <= 3; i++) {
+            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+            int port = 1000;
+            for (int j = 0; j < i; j++) {
+                SocketAddress address = new InetSocketAddress("127.0.0.1", port + j);
+                streamDistribution.put(address, new HashSet<String>());
+            }
+
+            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+            for (int k = 0; k < i + 1; k++) {
+                assertNull(chooser.choose());
+            }
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testSingleHostWithStreams() {
+        for (int i = 0; i < 3; i++) {
+            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+
+            Set<String> streams = new HashSet<String>();
+            for (int j = 0; j < 3; j++) {
+                streams.add("SingleHostStream-" + j);
+            }
+
+            int port = 1000;
+            SocketAddress address = new InetSocketAddress("127.0.0.1", port);
+            streamDistribution.put(address, streams);
+
+            for (int k = 1; k <= i; k++) {
+                address = new InetSocketAddress("127.0.0.1", port + k);
+                streamDistribution.put(address, new HashSet<String>());
+            }
+
+            Set<String> choosenStreams = new HashSet<String>();
+
+            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+            for (int l = 0; l < 3 + i + 1; l++) {
+                String s = chooser.choose();
+                if (null != s) {
+                    choosenStreams.add(s);
+                }
+            }
+
+            assertEquals(streams.size(), choosenStreams.size());
+            assertTrue(Sets.difference(streams, choosenStreams).immutableCopy().isEmpty());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testHostsHaveSameNumberStreams() {
+        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+        Set<String> allStreams = new HashSet<String>();
+
+        int numHosts = 3;
+        int numStreamsPerHost = 3;
+
+        int port = 1000;
+        for (int i = 1; i <= numHosts; i++) {
+            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
+            Set<String> streams = new HashSet<String>();
+
+            for (int j = 1; j <= numStreamsPerHost; j++) {
+                String streamName = "HostsHaveSameNumberStreams-" + i + "-" + j;
+                streams.add(streamName);
+                allStreams.add(streamName);
+            }
+
+            streamDistribution.put(address, streams);
+        }
+
+        Set<String> streamsChoosen = new HashSet<String>();
+        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+        for (int i = 1; i <= numStreamsPerHost; i++) {
+            for (int j = 1; j <= numHosts; j++) {
+                String s = chooser.choose();
+                assertNotNull(s);
+                streamsChoosen.add(s);
+            }
+            for (int j = 0; j < numHosts; j++) {
+                assertEquals(numStreamsPerHost - i, chooser.streamsDistribution.get(j).getRight().size());
+            }
+        }
+        assertNull(chooser.choose());
+        assertEquals(numHosts * numStreamsPerHost, streamsChoosen.size());
+        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testHostsHaveDifferentNumberStreams() {
+        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+        Set<String> allStreams = new HashSet<String>();
+
+        int numHosts = 6;
+        int maxStreamsPerHost = 4;
+
+        int port = 1000;
+        for (int i = 0; i < numHosts; i++) {
+            int group = i / 2;
+            int numStreamsThisGroup = maxStreamsPerHost - group;
+
+            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
+            Set<String> streams = new HashSet<String>();
+
+            for (int j = 1; j <= numStreamsThisGroup; j++) {
+                String streamName = "HostsHaveDifferentNumberStreams-" + i + "-" + j;
+                streams.add(streamName);
+                allStreams.add(streamName);
+            }
+
+            streamDistribution.put(address, streams);
+        }
+
+        Set<String> streamsChoosen = new HashSet<String>();
+        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+
+        for (int i = 0; i < allStreams.size(); i++) {
+            String s = chooser.choose();
+            assertNotNull(s);
+            streamsChoosen.add(s);
+        }
+        assertNull(chooser.choose());
+        assertEquals(allStreams.size(), streamsChoosen.size());
+        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testLimitedStreamChooser() {
+        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+
+        Set<String> streams = new HashSet<String>();
+        for (int j = 0; j < 10; j++) {
+            streams.add("SingleHostStream-" + j);
+        }
+
+        int port = 1000;
+        SocketAddress address = new InetSocketAddress("127.0.0.1", port);
+        streamDistribution.put(address, streams);
+
+        Set<String> choosenStreams = new HashSet<String>();
+
+        CountBasedStreamChooser underlying = new CountBasedStreamChooser(streamDistribution);
+        LimitedStreamChooser chooser = LimitedStreamChooser.of(underlying, 1);
+        for (int l = 0; l < 10; l++) {
+            String s = chooser.choose();
+            if (null != s) {
+                choosenStreams.add(s);
+            }
+        }
+
+        assertEquals(1, choosenStreams.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java
new file mode 100644
index 0000000..73fa98a
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.distributedlog.service.balancer;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.DistributedLogServerTestCase;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import java.util.Set;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link SimpleBalancer}.
+ */
+public class TestSimpleBalancer extends DistributedLogServerTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestSimpleBalancer.class);
+
+    DLClient targetClient;
+    DLServer targetServer;
+
+    public TestSimpleBalancer() {
+        super(true);
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        targetServer = createDistributedLogServer(7003);
+        targetClient = createDistributedLogClient("target", Optional.<String>absent());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != targetClient) {
+            targetClient.shutdown();
+        }
+        if (null != targetServer) {
+            targetServer.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBalanceAll() throws Exception {
+        String namePrefix = "simplebalancer-balance-all-";
+        int numStreams = 10;
+
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            // src client
+            dlClient.routingService.addHost(name, dlServer.getAddress());
+            // target client
+            targetClient.routingService.addHost(name, targetServer.getAddress());
+        }
+
+        // write to multiple streams
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
+        }
+
+        // validation
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+        }
+
+        Optional<RateLimiter> rateLimiter = Optional.absent();
+
+        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
+                                               "target", targetClient.dlClient, targetClient.dlClient);
+        logger.info("Rebalancing from 'unknown' target");
+        try {
+            balancer.balanceAll("unknown", 10, rateLimiter);
+            fail("Should fail on balanceAll from 'unknown' target.");
+        } catch (IllegalArgumentException iae) {
+            // expected
+        }
+
+        // nothing to balance from 'target'
+        logger.info("Rebalancing from 'target' target");
+        balancer.balanceAll("target", 1, rateLimiter);
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+        }
+
+        // balance all streams from 'source'
+        logger.info("Rebalancing from 'source' target");
+        balancer.balanceAll("source", 10, rateLimiter);
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, targetClient, targetServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBalanceStreams() throws Exception {
+        String namePrefix = "simplebalancer-balance-streams-";
+        int numStreams = 10;
+
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            // src client
+            dlClient.routingService.addHost(name, dlServer.getAddress());
+            // target client
+            targetClient.routingService.addHost(name, targetServer.getAddress());
+        }
+
+        // write to multiple streams
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
+        }
+
+        // validation
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+        }
+
+        Optional<RateLimiter> rateLimiter = Optional.absent();
+
+        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
+                                               "target", targetClient.dlClient, targetClient.dlClient);
+
+        // balance all streams from 'source'
+        logger.info("Rebalancing streams between targets");
+        balancer.balance(0, 0, 10, rateLimiter);
+
+        Set<String> sourceStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(dlClient));
+        Set<String> targetStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(targetClient));
+
+        assertEquals(numStreams / 2, sourceStreams.size());
+        assertEquals(numStreams / 2, targetStreams.size());
+
+        for (String name : sourceStreams) {
+            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, true, true);
+            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, false, false);
+        }
+
+        for (String name : targetStreams) {
+            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, true, true);
+            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, false, false);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
new file mode 100644
index 0000000..ce7b2c1
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.DistributedLogServerTestCase;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StreamMover}.
+ */
+public class TestStreamMover extends DistributedLogServerTestCase {
+
+    DLClient targetClient;
+    DLServer targetServer;
+
+    public TestStreamMover() {
+        super(true);
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        targetServer = createDistributedLogServer(7003);
+        targetClient = createDistributedLogClient("target", Optional.<String>absent());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != targetClient) {
+            targetClient.shutdown();
+        }
+        if (null != targetServer) {
+            targetServer.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testMoveStream() throws Exception {
+        String name = "dlserver-move-stream";
+
+        // src client
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+        // target client
+        targetClient.routingService.addHost(name, targetServer.getAddress());
+
+        // src client write a record to that stream
+        Await.result(((DistributedLogClient) dlClient.dlClient).write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+        checkStream(name, dlClient, dlServer, 1, 1, 1, true, true);
+        checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+
+        StreamMover streamMover = new StreamMoverImpl("source", dlClient.dlClient, dlClient.dlClient,
+                                                      "target", targetClient.dlClient, targetClient.dlClient);
+        assertTrue(streamMover.moveStream(name));
+        checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
+        checkStream(name, targetClient, targetServer, 1, 1, 1, true, true);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java
new file mode 100644
index 0000000..71dfa45
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.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.service.config;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServerConfiguration}.
+ */
+public class TestServerConfiguration {
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testUnassignedShardId() {
+        new ServerConfiguration().validate();
+    }
+
+    @Test(timeout = 60000)
+    public void testAssignedShardId() {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setServerShardId(100);
+        conf.validate();
+        assertEquals(100, conf.getServerShardId());
+    }
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testInvalidServerThreads() {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setServerShardId(100);
+        conf.setServerThreads(-1);
+        conf.validate();
+    }
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testInvalidDlsnVersion() {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setServerShardId(100);
+        conf.setDlsnVersion((byte) 9999);
+        conf.validate();
+    }
+
+    @Test(timeout = 60000)
+    public void testUseHostnameAsAllocatorPoolName() {
+        ServerConfiguration conf = new ServerConfiguration();
+        assertFalse("Should not use hostname by default", conf.isUseHostnameAsAllocatorPoolName());
+        conf.setUseHostnameAsAllocatorPoolName(true);
+        assertTrue("Should use hostname now", conf.isUseHostnameAsAllocatorPoolName());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
new file mode 100644
index 0000000..bdbde11
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.config;
+
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.config.PropertiesWriter;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StreamConfigProvider}.
+ */
+public class TestStreamConfigProvider {
+    private static final String DEFAULT_CONFIG_DIR = "conf";
+    private final String defaultConfigPath;
+    private final ScheduledExecutorService configExecutorService;
+
+    public TestStreamConfigProvider() throws Exception {
+        this.configExecutorService = Executors.newScheduledThreadPool(1,
+                new ThreadFactoryBuilder().setNameFormat("DistributedLogService-Dyncfg-%d").build());
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.save();
+        this.defaultConfigPath = writer.getFile().getPath();
+    }
+
+    StreamConfigProvider getServiceProvider(StreamPartitionConverter converter)
+            throws Exception {
+        return getServiceProvider(converter, DEFAULT_CONFIG_DIR);
+    }
+
+    StreamConfigProvider getServiceProvider(
+            StreamPartitionConverter converter,
+            String configPath,
+            String defaultPath) throws Exception {
+        return new ServiceStreamConfigProvider(
+                configPath,
+                defaultPath,
+                converter,
+                configExecutorService,
+                1,
+                TimeUnit.SECONDS);
+    }
+
+    StreamConfigProvider getServiceProvider(
+            StreamPartitionConverter converter,
+            String configPath) throws Exception {
+        return getServiceProvider(converter, configPath, defaultConfigPath);
+    }
+
+    StreamConfigProvider getDefaultProvider(String configFile) throws Exception {
+        return new DefaultStreamConfigProvider(configFile, configExecutorService, 1, TimeUnit.SECONDS);
+    }
+
+    StreamConfigProvider getNullProvider() throws Exception {
+        return new NullStreamConfigProvider();
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceProviderWithConfigRouters() throws Exception {
+        getServiceProvider(new IdentityStreamPartitionConverter());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceProviderWithMissingConfig() throws Exception {
+        StreamConfigProvider provider = getServiceProvider(new IdentityStreamPartitionConverter());
+        Optional<DynamicDistributedLogConfiguration> config = provider.getDynamicStreamConfig("stream1");
+        assertTrue(config.isPresent());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceProviderWithDefaultConfigPath() throws Exception {
+        // Default config with property set.
+        PropertiesWriter writer1 = new PropertiesWriter();
+        writer1.setProperty("rpsStreamAcquireServiceLimit", "191919");
+        writer1.save();
+        String fallbackConfPath1 = writer1.getFile().getPath();
+        StreamConfigProvider provider1 = getServiceProvider(new IdentityStreamPartitionConverter(),
+                DEFAULT_CONFIG_DIR, fallbackConfPath1);
+        Optional<DynamicDistributedLogConfiguration> config1 = provider1.getDynamicStreamConfig("stream1");
+
+        // Empty default config.
+        PropertiesWriter writer2 = new PropertiesWriter();
+        writer2.save();
+        String fallbackConfPath2 = writer2.getFile().getPath();
+        StreamConfigProvider provider2 = getServiceProvider(new IdentityStreamPartitionConverter(),
+                DEFAULT_CONFIG_DIR, fallbackConfPath2);
+        Optional<DynamicDistributedLogConfiguration> config2 = provider2.getDynamicStreamConfig("stream1");
+
+        assertEquals(191919, config1.get().getRpsStreamAcquireServiceLimit());
+        assertEquals(-1, config2.get().getRpsStreamAcquireServiceLimit());
+    }
+
+    @Test(timeout = 60000)
+    public void testDefaultProvider() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, "99");
+        writer.save();
+        StreamConfigProvider provider = getDefaultProvider(writer.getFile().getPath());
+        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
+        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
+        assertTrue(config1.isPresent());
+        assertTrue(config1.get() == config2.get());
+        assertEquals(99, config1.get().getRetentionPeriodHours());
+    }
+
+    @Test(timeout = 60000)
+    public void testNullProvider() throws Exception {
+        StreamConfigProvider provider = getNullProvider();
+        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
+        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
+        assertFalse(config1.isPresent());
+        assertTrue(config1 == config2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
new file mode 100644
index 0000000..5f5ecd4
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.LinkedHashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test Case for {@link LeastLoadPlacementPolicy}.
+ */
+public class TestLeastLoadPlacementPolicy {
+
+    @Test(timeout = 10000)
+    public void testCalculateBalances() throws Exception {
+        int numSevers = new Random().nextInt(20) + 1;
+        int numStreams = new Random().nextInt(200) + 1;
+        RoutingService mockRoutingService = mock(RoutingService.class);
+        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
+            new EqualLoadAppraiser(),
+            mockRoutingService,
+            mockNamespace,
+            null,
+            Duration.fromSeconds(600),
+            new NullStatsLogger());
+        TreeSet<ServerLoad> serverLoads =
+            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
+        long lowLoadPerServer = numStreams / numSevers;
+        long highLoadPerServer = lowLoadPerServer + 1;
+        for (ServerLoad serverLoad : serverLoads) {
+            long load = serverLoad.getLoad();
+            assertEquals(load, serverLoad.getStreamLoads().size());
+            assertTrue(String.format("Load %d is not between %d and %d",
+                load, lowLoadPerServer, highLoadPerServer), load == lowLoadPerServer || load == highLoadPerServer);
+        }
+    }
+
+    @Test(timeout = 10000)
+    public void testRefreshAndPlaceStream() throws Exception {
+        int numSevers = new Random().nextInt(20) + 1;
+        int numStreams = new Random().nextInt(200) + 1;
+        RoutingService mockRoutingService = mock(RoutingService.class);
+        when(mockRoutingService.getHosts()).thenReturn(generateSocketAddresses(numSevers));
+        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        try {
+            when(mockNamespace.getLogs()).thenReturn(generateStreams(numStreams).iterator());
+        } catch (IOException e) {
+            fail();
+        }
+        PlacementStateManager mockPlacementStateManager = mock(PlacementStateManager.class);
+        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
+            new EqualLoadAppraiser(),
+            mockRoutingService,
+            mockNamespace,
+            mockPlacementStateManager,
+            Duration.fromSeconds(600),
+            new NullStatsLogger());
+        leastLoadPlacementPolicy.refresh();
+
+        final ArgumentCaptor<TreeSet> captor = ArgumentCaptor.forClass(TreeSet.class);
+        verify(mockPlacementStateManager).saveOwnership(captor.capture());
+        TreeSet<ServerLoad> serverLoads = (TreeSet<ServerLoad>) captor.getValue();
+        ServerLoad next = serverLoads.first();
+        String serverPlacement = Await.result(leastLoadPlacementPolicy.placeStream("newstream1"));
+        assertEquals(next.getServer(), serverPlacement);
+    }
+
+    @Test(timeout = 10000)
+    public void testCalculateUnequalWeight() throws Exception {
+        int numSevers = new Random().nextInt(20) + 1;
+        int numStreams = new Random().nextInt(200) + 1;
+    /* use AtomicInteger to have a final object in answer method */
+        final AtomicInteger maxLoad = new AtomicInteger(Integer.MIN_VALUE);
+        RoutingService mockRoutingService = mock(RoutingService.class);
+        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        LoadAppraiser mockLoadAppraiser = mock(LoadAppraiser.class);
+        when(mockLoadAppraiser.getStreamLoad(anyString())).then(new Answer<Future<StreamLoad>>() {
+            @Override
+            public Future<StreamLoad> answer(InvocationOnMock invocationOnMock) throws Throwable {
+                int load = new Random().nextInt(100000);
+                if (load > maxLoad.get()) {
+                    maxLoad.set(load);
+                }
+                return Future.value(new StreamLoad(invocationOnMock.getArguments()[0].toString(), load));
+            }
+        });
+        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
+            mockLoadAppraiser,
+            mockRoutingService,
+            mockNamespace,
+            null,
+            Duration.fromSeconds(600),
+            new NullStatsLogger());
+        TreeSet<ServerLoad> serverLoads =
+            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
+        long highestLoadSeen = Long.MIN_VALUE;
+        long lowestLoadSeen = Long.MAX_VALUE;
+        for (ServerLoad serverLoad : serverLoads) {
+            long load = serverLoad.getLoad();
+            if (load < lowestLoadSeen) {
+                lowestLoadSeen = load;
+            }
+            if (load > highestLoadSeen) {
+                highestLoadSeen = load;
+            }
+        }
+        assertTrue("Unexpected placement for " + numStreams + " streams to "
+                + numSevers + " servers : highest load = " + highestLoadSeen
+                + ", lowest load = " + lowestLoadSeen + ", max stream load = " + maxLoad.get(),
+            highestLoadSeen - lowestLoadSeen <= maxLoad.get());
+    }
+
+    private Set<SocketAddress> generateSocketAddresses(int num) {
+        LinkedHashSet<SocketAddress> socketAddresses = new LinkedHashSet<SocketAddress>();
+        for (int i = 0; i < num; i++) {
+            socketAddresses.add(new InetSocketAddress(i));
+        }
+        return socketAddresses;
+    }
+
+    private Set<String> generateStreams(int num) {
+        LinkedHashSet<String> streams = new LinkedHashSet<String>();
+        for (int i = 0; i < num; i++) {
+            streams.add("stream_" + i);
+        }
+        return streams;
+    }
+
+    private Set<String> generateServers(int num) {
+        LinkedHashSet<String> servers = new LinkedHashSet<String>();
+        for (int i = 0; i < num; i++) {
+            servers.add("server_" + i);
+        }
+        return servers;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java
new file mode 100644
index 0000000..5bd234f
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServerLoad}.
+ */
+public class TestServerLoad {
+
+    @Test(timeout = 60000)
+    public void testSerializeDeserialize() throws IOException {
+        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
+        for (int i = 0; i < 20; i++) {
+            serverLoad.addStream(new StreamLoad("stream-" + i, i));
+        }
+        assertEquals(serverLoad, ServerLoad.deserialize(serverLoad.serialize()));
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLoad() throws IOException {
+        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
+        assertEquals(0, serverLoad.getLoad());
+        serverLoad.addStream(new StreamLoad("stream-" + 1, 3));
+        assertEquals(3, serverLoad.getLoad());
+        serverLoad.addStream(new StreamLoad("stream-" + 2, 7));
+        assertEquals(10, serverLoad.getLoad());
+        serverLoad.addStream(new StreamLoad("stream-" + 3, 1));
+        assertEquals(11, serverLoad.getLoad());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java
new file mode 100644
index 0000000..36a6fed
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StreamLoad}.
+ */
+public class TestStreamLoad {
+
+    @Test(timeout = 10000)
+    public void testSerializeDeserialize() throws IOException {
+        final String streamName = "aHellaRandomStreamName";
+        final int load = 1337;
+        final StreamLoad streamLoad = new StreamLoad(streamName, load);
+        assertEquals(streamLoad, StreamLoad.deserialize(streamLoad.serialize()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
new file mode 100644
index 0000000..07ec5a5
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.placement;
+
+import static org.apache.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import java.io.IOException;
+import java.net.URI;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.curator.test.TestingServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ZKPlacementStateManager}.
+ */
+public class TestZKPlacementStateManager {
+    private TestingServer zkTestServer;
+    private String zkServers;
+    private URI uri;
+    private ZKPlacementStateManager zkPlacementStateManager;
+
+    @Before
+    public void startZookeeper() throws Exception {
+        zkTestServer = new TestingServer(2181);
+        zkServers = "127.0.0.1:2181";
+        uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
+        zkPlacementStateManager =
+            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
+    }
+
+    @Test(timeout = 60000)
+    public void testSaveLoad() throws Exception {
+        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
+        zkPlacementStateManager.saveOwnership(ownerships);
+        SortedSet<ServerLoad> loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+
+        ownerships.add(new ServerLoad("emptyServer"));
+        zkPlacementStateManager.saveOwnership(ownerships);
+        loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+
+        ServerLoad sl1 = new ServerLoad("server1");
+        sl1.addStream(new StreamLoad("stream1", 3));
+        sl1.addStream(new StreamLoad("stream2", 4));
+        ServerLoad sl2 = new ServerLoad("server2");
+        sl2.addStream(new StreamLoad("stream3", 1));
+        ownerships.add(sl1);
+        ownerships.add(sl2);
+        zkPlacementStateManager.saveOwnership(ownerships);
+        loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+
+        loadedOwnerships.remove(sl1);
+        zkPlacementStateManager.saveOwnership(ownerships);
+        loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+    }
+
+    private TreeSet<ServerLoad> waitForServerLoadsNotificationAsc(
+        LinkedBlockingQueue<TreeSet<ServerLoad>> notificationQueue,
+        int expectedNumServerLoads) throws InterruptedException {
+        TreeSet<ServerLoad> notification = notificationQueue.take();
+        assertNotNull(notification);
+        while (notification.size() < expectedNumServerLoads) {
+            notification = notificationQueue.take();
+        }
+        assertEquals(expectedNumServerLoads, notification.size());
+        return notification;
+    }
+
+    @Test(timeout = 60000)
+    public void testWatchIndefinitely() throws Exception {
+        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
+        ownerships.add(new ServerLoad("server1"));
+        final LinkedBlockingQueue<TreeSet<ServerLoad>> serverLoadNotifications =
+            new LinkedBlockingQueue<TreeSet<ServerLoad>>();
+        PlacementStateManager.PlacementCallback callback = new PlacementStateManager.PlacementCallback() {
+            @Override
+            public void callback(TreeSet<ServerLoad> serverLoads) {
+                serverLoadNotifications.add(serverLoads);
+            }
+        };
+        zkPlacementStateManager.saveOwnership(ownerships); // need to initialize the zk path before watching
+        zkPlacementStateManager.watch(callback);
+        // cannot verify the callback here as it may call before the verify is called
+
+        zkPlacementStateManager.saveOwnership(ownerships);
+        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 1));
+
+        ServerLoad server2 = new ServerLoad("server2");
+        server2.addStream(new StreamLoad("hella-important-stream", 415));
+        ownerships.add(server2);
+        zkPlacementStateManager.saveOwnership(ownerships);
+        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 2));
+    }
+
+    @Test(timeout = 60000)
+    public void testZkFormatting() throws Exception {
+        final String server = "host/10.0.0.0:31351";
+        final String zkFormattedServer = "host--10.0.0.0:31351";
+        URI uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
+        ZKPlacementStateManager zkPlacementStateManager =
+            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
+        assertEquals(zkFormattedServer, zkPlacementStateManager.serverToZkFormat(server));
+        assertEquals(server, zkPlacementStateManager.zkFormatToServer(zkFormattedServer));
+    }
+
+    @After
+    public void stopZookeeper() throws IOException {
+        zkTestServer.stop();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
new file mode 100644
index 0000000..56e9483
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import com.twitter.util.Await;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Case for StreamManager.
+ */
+public class TestStreamManager {
+
+    @Rule
+    public TestName testName = new TestName();
+
+    ScheduledExecutorService mockExecutorService = mock(ScheduledExecutorService.class);
+
+    @Test(timeout = 60000)
+    public void testCollectionMethods() throws Exception {
+        Stream mockStream = mock(Stream.class);
+        when(mockStream.getStreamName()).thenReturn("stream1");
+        when(mockStream.getPartition()).thenReturn(new Partition("stream1", 0));
+        StreamFactory mockStreamFactory = mock(StreamFactory.class);
+        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
+        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
+        when(mockStreamFactory.create(
+                (String) any(),
+                (DynamicDistributedLogConfiguration) any(),
+                (StreamManager) any())).thenReturn(mockStream);
+        StreamManager streamManager = new StreamManagerImpl(
+                "",
+                new DistributedLogConfiguration(),
+                mockExecutorService,
+                mockStreamFactory,
+                mockPartitionConverter,
+                mockStreamConfigProvider,
+                mock(DistributedLogNamespace.class));
+
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyAcquired(mockStream);
+        assertTrue(streamManager.isAcquired("stream1"));
+        assertEquals(1, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyReleased(mockStream);
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyAcquired(mockStream);
+        assertTrue(streamManager.isAcquired("stream1"));
+        assertEquals(1, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyAcquired(mockStream);
+        assertTrue(streamManager.isAcquired("stream1"));
+        assertEquals(1, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyReleased(mockStream);
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyReleased(mockStream);
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateStream() throws Exception {
+        Stream mockStream = mock(Stream.class);
+        final String streamName = "stream1";
+        when(mockStream.getStreamName()).thenReturn(streamName);
+        StreamFactory mockStreamFactory = mock(StreamFactory.class);
+        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
+        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
+        when(mockStreamFactory.create(
+            (String) any(),
+            (DynamicDistributedLogConfiguration) any(),
+            (StreamManager) any())
+        ).thenReturn(mockStream);
+        DistributedLogNamespace dlNamespace = mock(DistributedLogNamespace.class);
+        ScheduledExecutorService executorService = new ScheduledThreadPoolExecutor(1);
+
+        StreamManager streamManager = new StreamManagerImpl(
+                "",
+                new DistributedLogConfiguration(),
+                executorService,
+                mockStreamFactory,
+                mockPartitionConverter,
+                mockStreamConfigProvider,
+                dlNamespace);
+
+        assertTrue(Await.ready(streamManager.createStreamAsync(streamName)).isReturn());
+        verify(dlNamespace).createLog(streamName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
new file mode 100644
index 0000000..a18fda1
--- /dev/null
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.stream;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.acl.DefaultAccessControlManager;
+import org.apache.distributedlog.exceptions.InternalServerException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import java.nio.ByteBuffer;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Case for StreamOps.
+ */
+public class TestStreamOp {
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private WriteOp getWriteOp() {
+        SettableFeature disabledFeature = new SettableFeature("", 0);
+        return new WriteOp("test",
+            ByteBuffer.wrap("test".getBytes()),
+            new NullStatsLogger(),
+            new NullStatsLogger(),
+            new IdentityStreamPartitionConverter(),
+            new ServerConfiguration(),
+            (byte) 0,
+            null,
+            false,
+            disabledFeature,
+            DefaultAccessControlManager.INSTANCE);
+    }
+
+    @Test(timeout = 60000)
+    public void testResponseFailedTwice() throws Exception {
+        WriteOp writeOp = getWriteOp();
+        writeOp.fail(new InternalServerException("test1"));
+        writeOp.fail(new InternalServerException("test2"));
+
+        WriteResponse response = Await.result(writeOp.result());
+        assertEquals(StatusCode.INTERNAL_SERVER_ERROR, response.getHeader().getCode());
+        assertEquals(ResponseUtils.exceptionToHeader(new InternalServerException("test1")), response.getHeader());
+    }
+
+    @Test(timeout = 60000)
+    public void testResponseSucceededThenFailed() throws Exception {
+        AsyncLogWriter writer = mock(AsyncLogWriter.class);
+        when(writer.write((LogRecord) any())).thenReturn(Future.value(new DLSN(1, 2, 3)));
+        when(writer.getStreamName()).thenReturn("test");
+        WriteOp writeOp = getWriteOp();
+        writeOp.execute(writer, new Sequencer() {
+            public long nextId() {
+                return 0;
+            }
+        }, new Object());
+        writeOp.fail(new InternalServerException("test2"));
+
+        WriteResponse response = Await.result(writeOp.result());
+        assertEquals(StatusCode.SUCCESS, response.getHeader().getCode());
+    }
+}


[27/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 4fe8141..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 77b7beb..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 753a1af..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 2fc8de0..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 1ff7c93..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 352d755..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.routing;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 93cdf7a..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 38a7544..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.serverset;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index f1da33c..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index faf45c2..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 68fe8b0..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 4bdd4b1..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.speculative;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index c2dcddd..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 530c632..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 7a49faa..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index b94b4be..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 110e99a..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 106d3fc..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 org.apache.distributedlog.client.stats;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 68e6825..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 9f30815..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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();
-}


[20/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
new file mode 100644
index 0000000..86d1c11
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
new file mode 100644
index 0000000..8ef33bd
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
new file mode 100644
index 0000000..e38c2ed
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
new file mode 100644
index 0000000..ad1c878
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
new file mode 100644
index 0000000..6d9a471
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
new file mode 100644
index 0000000..f44cddd
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
new file mode 100644
index 0000000..59665b9
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
new file mode 100644
index 0000000..151663e
--- /dev/null
+++ b/distributedlog-proxy-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
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
new file mode 100644
index 0000000..d2d61a9
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..ab0cb58
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java b/distributedlog-proxy-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
new file mode 100644
index 0000000..d2df9a5
--- /dev/null
+++ b/distributedlog-proxy-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/c44e0278/distributedlog-proxy-client/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-client/src/test/resources/log4j.properties b/distributedlog-proxy-client/src/test/resources/log4j.properties
new file mode 100644
index 0000000..3e51059
--- /dev/null
+++ b/distributedlog-proxy-client/src/test/resources/log4j.properties
@@ -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.
+# */
+
+#
+# DisributedLog Logging Configuration
+#
+
+# Example with rolling log file
+log4j.rootLogger=INFO, CONSOLE
+
+#disable zookeeper logging
+log4j.logger.org.apache.zookeeper=OFF
+#Set the bookkeeper level to warning
+log4j.logger.org.apache.bookkeeper=INFO
+
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.Threshold=INFO
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+# Add ROLLINGFILE to rootLogger to get log file output
+#    Log DEBUG level and above messages to a log file
+#log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
+#log4j.appender.ROLLINGFILE.Threshold=INFO
+#log4j.appender.ROLLINGFILE.File=distributedlog.log
+#log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+#log4j.appender.ROLLINGFILE.DatePattern='.'yyyy-MM-dd-HH-mm
+#log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+log4j.appender.R=org.apache.log4j.RollingFileAppender
+log4j.appender.R.Threshold=TRACE
+log4j.appender.R.File=target/error.log
+log4j.appender.R.MaxFileSize=200MB
+log4j.appender.R.MaxBackupIndex=7
+log4j.appender.R.layout=org.apache.log4j.PatternLayout
+log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n


[17/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java
new file mode 100644
index 0000000..b1e2879
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java
@@ -0,0 +1,469 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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 com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.Stat;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration;
+import com.twitter.util.FutureEventListener;
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.util.ArrayList;
+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.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Monitor Service.
+ */
+public class MonitorService implements NamespaceListener {
+
+    private static final Logger logger = LoggerFactory.getLogger(MonitorService.class);
+
+    private DistributedLogNamespace dlNamespace = null;
+    private MonitorServiceClient dlClient = null;
+    private DLZkServerSet[] zkServerSets = null;
+    private final ScheduledExecutorService executorService =
+            Executors.newScheduledThreadPool(Runtime.getRuntime().availableProcessors());
+    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+    private final Map<String, StreamChecker> knownStreams = new HashMap<String, StreamChecker>();
+
+    // Settings
+    private int regionId = DistributedLogConstants.LOCAL_REGION_ID;
+    private int interval = 100;
+    private String streamRegex = null;
+    private boolean watchNamespaceChanges = false;
+    private boolean handshakeWithClientInfo = false;
+    private int heartbeatEveryChecks = 0;
+    private int instanceId = -1;
+    private int totalInstances = -1;
+    private boolean isThriftMux = false;
+
+    // Options
+    private final Optional<String> uriArg;
+    private final Optional<String> confFileArg;
+    private final Optional<String> serverSetArg;
+    private final Optional<Integer> intervalArg;
+    private final Optional<Integer> regionIdArg;
+    private final Optional<String> streamRegexArg;
+    private final Optional<Integer> instanceIdArg;
+    private final Optional<Integer> totalInstancesArg;
+    private final Optional<Integer> heartbeatEveryChecksArg;
+    private final Optional<Boolean> handshakeWithClientInfoArg;
+    private final Optional<Boolean> watchNamespaceChangesArg;
+    private final Optional<Boolean> isThriftMuxArg;
+
+    // Stats
+    private final StatsProvider statsProvider;
+    private final StatsReceiver statsReceiver;
+    private final StatsReceiver monitorReceiver;
+    private final Stat successStat;
+    private final Stat failureStat;
+    private final Gauge<Number> numOfStreamsGauge;
+    // Hash Function
+    private final HashFunction hashFunction = Hashing.md5();
+
+    class StreamChecker implements Runnable, FutureEventListener<Void>, LogSegmentListener {
+        private final String name;
+        private volatile boolean closed = false;
+        private volatile boolean checking = false;
+        private final Stopwatch stopwatch = Stopwatch.createUnstarted();
+        private DistributedLogManager dlm = null;
+        private int numChecks = 0;
+
+        StreamChecker(String name) {
+            this.name = name;
+        }
+
+        @Override
+        public void run() {
+            if (null == dlm) {
+                try {
+                    dlm = dlNamespace.openLog(name);
+                    dlm.registerListener(this);
+                } catch (IOException e) {
+                    if (null != dlm) {
+                        try {
+                            dlm.close();
+                        } catch (IOException e1) {
+                            logger.error("Failed to close dlm for {} : ", name, e1);
+                        }
+                        dlm = null;
+                    }
+                    executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
+                }
+            } else {
+                stopwatch.reset().start();
+                boolean sendHeartBeat;
+                if (heartbeatEveryChecks > 0) {
+                    synchronized (this) {
+                        ++numChecks;
+                        if (numChecks >= Integer.MAX_VALUE) {
+                            numChecks = 0;
+                        }
+                        sendHeartBeat = (numChecks % heartbeatEveryChecks) == 0;
+                    }
+                } else {
+                    sendHeartBeat = false;
+                }
+                if (sendHeartBeat) {
+                    dlClient.heartbeat(name).addEventListener(this);
+                } else {
+                    dlClient.check(name).addEventListener(this);
+                }
+            }
+        }
+
+        @Override
+        public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
+            if (segments.size() > 0 && segments.get(0).getRegionId() == regionId) {
+                if (!checking) {
+                    logger.info("Start checking stream {}.", name);
+                    checking = true;
+                    run();
+                }
+            } else {
+                if (checking) {
+                    logger.info("Stop checking stream {}.", name);
+                }
+            }
+        }
+
+        @Override
+        public void onLogStreamDeleted() {
+            logger.info("Stream {} is deleted", name);
+        }
+
+        @Override
+        public void onSuccess(Void value) {
+            successStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            scheduleCheck();
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            failureStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            scheduleCheck();
+        }
+
+        private void scheduleCheck() {
+            if (closed) {
+                return;
+            }
+            if (!checking) {
+                return;
+            }
+            try {
+                executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
+            } catch (RejectedExecutionException ree) {
+                logger.error("Failed to schedule checking stream {} in {} ms : ",
+                        new Object[] { name, interval, ree });
+            }
+        }
+
+        private void close() {
+            closed = true;
+            if (null != dlm) {
+                try {
+                    dlm.close();
+                } catch (IOException e) {
+                    logger.error("Failed to close dlm for {} : ", name, e);
+                }
+            }
+        }
+    }
+
+    MonitorService(Optional<String> uriArg,
+                   Optional<String> confFileArg,
+                   Optional<String> serverSetArg,
+                   Optional<Integer> intervalArg,
+                   Optional<Integer> regionIdArg,
+                   Optional<String> streamRegexArg,
+                   Optional<Integer> instanceIdArg,
+                   Optional<Integer> totalInstancesArg,
+                   Optional<Integer> heartbeatEveryChecksArg,
+                   Optional<Boolean> handshakeWithClientInfoArg,
+                   Optional<Boolean> watchNamespaceChangesArg,
+                   Optional<Boolean> isThriftMuxArg,
+                   StatsReceiver statsReceiver,
+                   StatsProvider statsProvider) {
+        // options
+        this.uriArg = uriArg;
+        this.confFileArg = confFileArg;
+        this.serverSetArg = serverSetArg;
+        this.intervalArg = intervalArg;
+        this.regionIdArg = regionIdArg;
+        this.streamRegexArg = streamRegexArg;
+        this.instanceIdArg = instanceIdArg;
+        this.totalInstancesArg = totalInstancesArg;
+        this.heartbeatEveryChecksArg = heartbeatEveryChecksArg;
+        this.handshakeWithClientInfoArg = handshakeWithClientInfoArg;
+        this.watchNamespaceChangesArg = watchNamespaceChangesArg;
+        this.isThriftMuxArg = isThriftMuxArg;
+
+        // Stats
+        this.statsReceiver = statsReceiver;
+        this.monitorReceiver = statsReceiver.scope("monitor");
+        this.successStat = monitorReceiver.stat0("success");
+        this.failureStat = monitorReceiver.stat0("failure");
+        this.statsProvider = statsProvider;
+        this.numOfStreamsGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return knownStreams.size();
+            }
+        };
+    }
+
+    public void runServer() throws IllegalArgumentException, IOException {
+        checkArgument(uriArg.isPresent(),
+                "No distributedlog uri provided.");
+        checkArgument(serverSetArg.isPresent(),
+                "No proxy server set provided.");
+        if (intervalArg.isPresent()) {
+            interval = intervalArg.get();
+        }
+        if (regionIdArg.isPresent()) {
+            regionId = regionIdArg.get();
+        }
+        if (streamRegexArg.isPresent()) {
+            streamRegex = streamRegexArg.get();
+        }
+        if (instanceIdArg.isPresent()) {
+            instanceId = instanceIdArg.get();
+        }
+        if (totalInstancesArg.isPresent()) {
+            totalInstances = totalInstancesArg.get();
+        }
+        if (heartbeatEveryChecksArg.isPresent()) {
+            heartbeatEveryChecks = heartbeatEveryChecksArg.get();
+        }
+        if (instanceId < 0 || totalInstances <= 0 || instanceId >= totalInstances) {
+            throw new IllegalArgumentException("Invalid instance id or total instances number.");
+        }
+        handshakeWithClientInfo = handshakeWithClientInfoArg.isPresent();
+        watchNamespaceChanges = watchNamespaceChangesArg.isPresent();
+        isThriftMux = isThriftMuxArg.isPresent();
+        URI uri = URI.create(uriArg.get());
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+        if (confFileArg.isPresent()) {
+            String configFile = confFileArg.get();
+            try {
+                dlConf.loadConf(new File(configFile).toURI().toURL());
+            } catch (ConfigurationException e) {
+                throw new IOException("Failed to load distributedlog configuration from " + configFile + ".");
+            } catch (MalformedURLException e) {
+                throw new IOException("Failed to load distributedlog configuration from malformed "
+                        + configFile + ".");
+            }
+        }
+        logger.info("Starting stats provider : {}.", statsProvider.getClass());
+        statsProvider.start(dlConf);
+        String[] serverSetPaths = StringUtils.split(serverSetArg.get(), ",");
+        if (serverSetPaths.length == 0) {
+            throw new IllegalArgumentException("Invalid serverset paths provided : " + serverSetArg.get());
+        }
+
+        ServerSet[] serverSets = createServerSets(serverSetPaths);
+        ServerSet local = serverSets[0];
+        ServerSet[] remotes  = new ServerSet[serverSets.length - 1];
+        System.arraycopy(serverSets, 1, remotes, 0, remotes.length);
+
+        ClientBuilder finagleClientBuilder = ClientBuilder.get()
+            .connectTimeout(Duration.fromSeconds(1))
+            .tcpConnectTimeout(Duration.fromSeconds(1))
+            .requestTimeout(Duration.fromSeconds(2))
+            .keepAlive(true)
+            .failFast(false);
+
+        if (!isThriftMux) {
+            finagleClientBuilder = finagleClientBuilder
+                .hostConnectionLimit(2)
+                .hostConnectionCoresize(2);
+        }
+
+        dlClient = DistributedLogClientBuilder.newBuilder()
+                .name("monitor")
+                .thriftmux(isThriftMux)
+                .clientId(ClientId$.MODULE$.apply("monitor"))
+                .redirectBackoffMaxMs(50)
+                .redirectBackoffStartMs(100)
+                .requestTimeoutMs(2000)
+                .maxRedirects(2)
+                .serverSets(local, remotes)
+                .streamNameRegex(streamRegex)
+                .handshakeWithClientInfo(handshakeWithClientInfo)
+                .clientBuilder(finagleClientBuilder)
+                .statsReceiver(monitorReceiver.scope("client"))
+                .buildMonitorClient();
+        runMonitor(dlConf, uri);
+    }
+
+    ServerSet[] createServerSets(String[] serverSetPaths) {
+        ServerSet[] serverSets = new ServerSet[serverSetPaths.length];
+        zkServerSets = new DLZkServerSet[serverSetPaths.length];
+        for (int i = 0; i < serverSetPaths.length; i++) {
+            String serverSetPath = serverSetPaths[i];
+            zkServerSets[i] = parseServerSet(serverSetPath);
+            serverSets[i] = zkServerSets[i].getServerSet();
+        }
+        return serverSets;
+    }
+
+    protected DLZkServerSet parseServerSet(String serverSetPath) {
+        return DLZkServerSet.of(URI.create(serverSetPath), 60000);
+    }
+
+    @Override
+    public void onStreamsChanged(Iterator<String> streams) {
+        Set<String> newSet = new HashSet<String>();
+        while (streams.hasNext()) {
+            String s = streams.next();
+            if (null == streamRegex || s.matches(streamRegex)) {
+                if (Math.abs(hashFunction.hashUnencodedChars(s).asInt()) % totalInstances == instanceId) {
+                    newSet.add(s);
+                }
+            }
+        }
+        List<StreamChecker> tasksToCancel = new ArrayList<StreamChecker>();
+        synchronized (knownStreams) {
+            Set<String> knownStreamSet = new HashSet<String>(knownStreams.keySet());
+            Set<String> removedStreams = Sets.difference(knownStreamSet, newSet).immutableCopy();
+            Set<String> addedStreams = Sets.difference(newSet, knownStreamSet).immutableCopy();
+            for (String s : removedStreams) {
+                StreamChecker task = knownStreams.remove(s);
+                if (null != task) {
+                    logger.info("Removed stream {}", s);
+                    tasksToCancel.add(task);
+                }
+            }
+            for (String s : addedStreams) {
+                if (!knownStreams.containsKey(s)) {
+                    logger.info("Added stream {}", s);
+                    StreamChecker sc = new StreamChecker(s);
+                    knownStreams.put(s, sc);
+                    sc.run();
+                }
+            }
+        }
+        for (StreamChecker sc : tasksToCancel) {
+            sc.close();
+        }
+    }
+
+    void runMonitor(DistributedLogConfiguration conf, URI dlUri) throws IOException {
+        // stats
+        statsProvider.getStatsLogger("monitor").registerGauge("num_streams", numOfStreamsGauge);
+        logger.info("Construct dl namespace @ {}", dlUri);
+        dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(dlUri)
+                .build();
+        if (watchNamespaceChanges) {
+            dlNamespace.registerNamespaceListener(this);
+        } else {
+            onStreamsChanged(dlNamespace.getLogs());
+        }
+    }
+
+    /**
+     * Close the server.
+     */
+    public void close() {
+        logger.info("Closing monitor service.");
+        if (null != dlClient) {
+            dlClient.close();
+        }
+        if (null != zkServerSets) {
+            for (DLZkServerSet zkServerSet : zkServerSets) {
+                zkServerSet.close();
+            }
+        }
+        if (null != dlNamespace) {
+            dlNamespace.close();
+        }
+        executorService.shutdown();
+        try {
+            if (!executorService.awaitTermination(1, TimeUnit.MINUTES)) {
+                executorService.shutdownNow();
+            }
+        } catch (InterruptedException e) {
+            logger.error("Interrupted on waiting shutting down monitor executor service : ", e);
+        }
+        if (null != statsProvider) {
+            // clean up the gauges
+            unregisterGauge();
+            statsProvider.stop();
+        }
+        keepAliveLatch.countDown();
+        logger.info("Closed monitor service.");
+    }
+
+    public void join() throws InterruptedException {
+        keepAliveLatch.await();
+    }
+
+    /**
+     * clean up the gauge before we close to help GC.
+     */
+    private void unregisterGauge(){
+        statsProvider.getStatsLogger("monitor").unregisterGauge("num_streams", numOfStreamsGauge);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java
new file mode 100644
index 0000000..1f45b13
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.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.service;
+
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalStringArg;
+
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.io.IOException;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+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.cli.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The launcher to run monitor service.
+ */
+public class MonitorServiceApp {
+
+    private static final Logger logger = LoggerFactory.getLogger(MonitorServiceApp.class);
+
+    static final String USAGE = "MonitorService [-u <uri>] [-c <conf>] [-s serverset]";
+
+    final String[] args;
+    final Options options = new Options();
+
+    private MonitorServiceApp(String[] args) {
+        this.args = args;
+        // prepare options
+        options.addOption("u", "uri", true, "DistributedLog URI");
+        options.addOption("c", "conf", true, "DistributedLog Configuration File");
+        options.addOption("s", "serverset", true, "Proxy Server Set");
+        options.addOption("i", "interval", true, "Check interval");
+        options.addOption("d", "region", true, "Region ID");
+        options.addOption("p", "provider", true, "DistributedLog Stats Provider");
+        options.addOption("f", "filter", true, "Filter streams by regex");
+        options.addOption("w", "watch", false, "Watch stream changes under a given namespace");
+        options.addOption("n", "instance_id", true, "Instance ID");
+        options.addOption("t", "total_instances", true, "Total instances");
+        options.addOption("hck", "heartbeat-num-checks", true, "Send a heartbeat after num checks");
+        options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
+    }
+
+    void printUsage() {
+        HelpFormatter helpFormatter = new HelpFormatter();
+        helpFormatter.printHelp(USAGE, options);
+    }
+
+    private void run() {
+        try {
+            logger.info("Running monitor service.");
+            BasicParser parser = new BasicParser();
+            CommandLine cmdline = parser.parse(options, args);
+            runCmd(cmdline);
+        } catch (ParseException pe) {
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (IOException ie) {
+            logger.error("Failed to start monitor service : ", ie);
+            Runtime.getRuntime().exit(-1);
+        }
+    }
+
+    void runCmd(CommandLine cmdline) throws IOException {
+        StatsProvider statsProvider = new NullStatsProvider();
+        if (cmdline.hasOption("p")) {
+            String providerClass = cmdline.getOptionValue("p");
+            statsProvider = ReflectionUtils.newInstance(providerClass, StatsProvider.class);
+        }
+        StatsReceiver statsReceiver = NullStatsReceiver.get();
+
+        final MonitorService monitorService = new MonitorService(
+                getOptionalStringArg(cmdline, "u"),
+                getOptionalStringArg(cmdline, "c"),
+                getOptionalStringArg(cmdline, "s"),
+                getOptionalIntegerArg(cmdline, "i"),
+                getOptionalIntegerArg(cmdline, "d"),
+                getOptionalStringArg(cmdline, "f"),
+                getOptionalIntegerArg(cmdline, "n"),
+                getOptionalIntegerArg(cmdline, "t"),
+                getOptionalIntegerArg(cmdline, "hck"),
+                getOptionalBooleanArg(cmdline, "hsci"),
+                getOptionalBooleanArg(cmdline, "w"),
+                getOptionalBooleanArg(cmdline, "mx"),
+                statsReceiver,
+                statsProvider);
+
+        monitorService.runServer();
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                logger.info("Closing monitor service.");
+                monitorService.close();
+                logger.info("Closed monitor service.");
+            }
+        });
+        try {
+            monitorService.join();
+        } catch (InterruptedException ie) {
+            logger.warn("Interrupted when waiting monitor service to be finished : ", ie);
+        }
+    }
+
+    public static void main(String[] args) {
+        final MonitorServiceApp launcher = new MonitorServiceApp(args);
+        launcher.run();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
new file mode 100644
index 0000000..08f4b41
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.exceptions.DLException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+
+/**
+ * Utility methods for building write proxy service responses.
+ */
+public class ResponseUtils {
+    public static ResponseHeader deniedHeader() {
+        return new ResponseHeader(StatusCode.REQUEST_DENIED);
+    }
+
+    public static ResponseHeader streamUnavailableHeader() {
+        return new ResponseHeader(StatusCode.STREAM_UNAVAILABLE);
+    }
+
+    public static ResponseHeader successHeader() {
+        return new ResponseHeader(StatusCode.SUCCESS);
+    }
+
+    public static ResponseHeader ownerToHeader(String owner) {
+        return new ResponseHeader(StatusCode.FOUND).setLocation(owner);
+    }
+
+    public static ResponseHeader exceptionToHeader(Throwable t) {
+        ResponseHeader response = new ResponseHeader();
+        if (t instanceof DLException) {
+            DLException dle = (DLException) t;
+            if (dle instanceof OwnershipAcquireFailedException) {
+                response.setLocation(((OwnershipAcquireFailedException) dle).getCurrentOwner());
+            }
+            response.setCode(StatusCode.findByValue(dle.getCode()));
+            response.setErrMsg(dle.getMessage());
+        } else {
+            response.setCode(StatusCode.INTERNAL_SERVER_ERROR);
+            response.setErrMsg("Internal server error : " + t.getMessage());
+        }
+        return response;
+    }
+
+    public static WriteResponse write(ResponseHeader responseHeader) {
+        return new WriteResponse(responseHeader);
+    }
+
+    public static WriteResponse writeSuccess() {
+        return new WriteResponse(successHeader());
+    }
+
+    public static WriteResponse writeDenied() {
+        return new WriteResponse(deniedHeader());
+    }
+
+    public static BulkWriteResponse bulkWrite(ResponseHeader responseHeader) {
+        return new BulkWriteResponse(responseHeader);
+    }
+
+    public static BulkWriteResponse bulkWriteSuccess() {
+        return new BulkWriteResponse(successHeader());
+    }
+
+    public static BulkWriteResponse bulkWriteDenied() {
+        return new BulkWriteResponse(deniedHeader());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java
new file mode 100644
index 0000000..436145d
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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;
+
+/**
+ * List of feature keys used by distributedlog server.
+ */
+public enum ServerFeatureKeys {
+
+    REGION_STOP_ACCEPT_NEW_STREAM,
+    SERVICE_RATE_LIMIT_DISABLED,
+    SERVICE_CHECKSUM_DISABLED,
+    SERVICE_GLOBAL_LIMITER_DISABLED
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/StatsFilter.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/StatsFilter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/StatsFilter.java
new file mode 100644
index 0000000..ee64580
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/StatsFilter.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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 com.google.common.base.Stopwatch;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.SimpleFilter;
+import com.twitter.util.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Track distributedlog server finagle-service stats.
+ */
+class StatsFilter<Req, Rep> extends SimpleFilter<Req, Rep> {
+
+    private final StatsLogger stats;
+    private final Counter outstandingAsync;
+    private final OpStatsLogger serviceExec;
+
+    @Override
+    public Future<Rep> apply(Req req, Service<Req, Rep> service) {
+        Future<Rep> result = null;
+        outstandingAsync.inc();
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        try {
+            result = service.apply(req);
+            serviceExec.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+        } finally {
+            outstandingAsync.dec();
+            if (null == result) {
+                serviceExec.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+        }
+        return result;
+    }
+
+    public StatsFilter(StatsLogger stats) {
+        this.stats = stats;
+        this.outstandingAsync = stats.getCounter("outstandingAsync");
+        this.serviceExec = stats.getOpStatsLogger("serviceExec");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java
new file mode 100644
index 0000000..ee64fc7
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.announcer;
+
+import java.io.IOException;
+
+/**
+ * Announce service information.
+ */
+public interface Announcer {
+
+    /**
+     * Announce service info.
+     */
+    void announce() throws IOException;
+
+    /**
+     * Unannounce the service info.
+     */
+    void unannounce() throws IOException;
+
+    /**
+     * Close the announcer.
+     */
+    void close();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java
new file mode 100644
index 0000000..5a1277a
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.announcer;
+
+import java.io.IOException;
+
+/**
+ * A no-op implementation of {@link Announcer}.
+ */
+public class NOPAnnouncer implements Announcer {
+    @Override
+    public void announce() throws IOException {
+        // nop
+    }
+
+    @Override
+    public void unannounce() throws IOException {
+        // nop
+    }
+
+    @Override
+    public void close() {
+        // nop
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
new file mode 100644
index 0000000..df4a8e2
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.announcer;
+
+import com.twitter.common.zookeeper.Group;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ServerSet based announcer.
+ */
+public class ServerSetAnnouncer implements Announcer {
+
+    private static final Logger logger = LoggerFactory.getLogger(ServerSetAnnouncer.class);
+
+    final String localAddr;
+    final InetSocketAddress serviceEndpoint;
+    final Map<String, InetSocketAddress> additionalEndpoints;
+    final int shardId;
+
+    // ServerSet
+    DLZkServerSet zkServerSet;
+
+    // Service Status
+    ServerSet.EndpointStatus serviceStatus = null;
+
+    /**
+     * Announce server infos.
+     *
+     * @param servicePort
+     *          service port
+     * @param statsPort
+     *          stats port
+     * @param shardId
+     *          shard id
+     */
+    public ServerSetAnnouncer(URI uri,
+                              int servicePort,
+                              int statsPort,
+                              int shardId) throws UnknownHostException {
+        this.shardId = shardId;
+        this.localAddr = InetAddress.getLocalHost().getHostAddress();
+        // service endpoint
+        this.serviceEndpoint = new InetSocketAddress(localAddr, servicePort);
+        // stats endpoint
+        InetSocketAddress statsEndpoint = new InetSocketAddress(localAddr, statsPort);
+        this.additionalEndpoints = new HashMap<String, InetSocketAddress>();
+        this.additionalEndpoints.put("aurora", statsEndpoint);
+        this.additionalEndpoints.put("stats", statsEndpoint);
+        this.additionalEndpoints.put("service", serviceEndpoint);
+        this.additionalEndpoints.put("thrift", serviceEndpoint);
+
+        // Create zookeeper and server set
+        this.zkServerSet = DLZkServerSet.of(uri, 60000);
+    }
+
+    @Override
+    public synchronized void announce() throws IOException {
+        try {
+            serviceStatus =
+                    zkServerSet.getServerSet().join(serviceEndpoint, additionalEndpoints, shardId);
+        } catch (Group.JoinException e) {
+            throw new IOException("Failed to announce service : ", e);
+        } catch (InterruptedException e) {
+            logger.warn("Interrupted on announcing service : ", e);
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public synchronized void unannounce() throws IOException {
+        if (null == serviceStatus) {
+            logger.warn("No service to unannounce.");
+            return;
+        }
+        try {
+            serviceStatus.leave();
+        } catch (ServerSet.UpdateException e) {
+            throw new IOException("Failed to unannounce service : ", e);
+        }
+    }
+
+    @Override
+    public void close() {
+        zkServerSet.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/package-info.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/package-info.java
new file mode 100644
index 0000000..6559bb3
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/announcer/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.
+ */
+/**
+ * Announcers to announce servers to server set.
+ */
+package org.apache.distributedlog.service.announcer;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java
new file mode 100644
index 0000000..cdffaa3
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/Balancer.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.service.balancer;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+
+/**
+ * Balancer Interface.
+ *
+ * <p>A balancer is used for balance the streams across the proxy cluster.
+ */
+public interface Balancer {
+
+    /**
+     * Rebalance all the streams from <i>source</i> to others.
+     *
+     * @param source
+     *          source target name.
+     * @param rebalanceConcurrency
+     *          the concurrency to move streams for re-balance.
+     * @param rebalanceRateLimiter
+     *          the rate limiting to move streams for re-balance.
+     */
+    void balanceAll(String source,
+                    int rebalanceConcurrency,
+                    Optional<RateLimiter> rebalanceRateLimiter);
+
+    /**
+     * Balance the streams across all targets.
+     *
+     * @param rebalanceWaterMark
+     *          rebalance water mark. if number of streams of a given target is less than
+     *          the water mark, no streams will be re-balanced from this target.
+     * @param rebalanceTolerancePercentage
+     *          tolerance percentage for the balancer. if number of streams of a given target is
+     *          less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
+     *          be re-balanced from that target.
+     * @param rebalanceConcurrency
+     *          the concurrency to move streams for re-balance.
+     * @param rebalanceRateLimiter
+     *          the rate limiting to move streams for re-balance.
+     */
+    void balance(int rebalanceWaterMark,
+                 double rebalanceTolerancePercentage,
+                 int rebalanceConcurrency,
+                 Optional<RateLimiter> rebalanceRateLimiter);
+
+    /**
+     * Close the balancer.
+     */
+    void close();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
new file mode 100644
index 0000000..964c1cc
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
@@ -0,0 +1,327 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.service.ClientUtils;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.tools.Tool;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tool to rebalance cluster.
+ */
+public class BalancerTool extends Tool {
+
+    private static final Logger logger = LoggerFactory.getLogger(BalancerTool.class);
+
+    static DistributedLogClientBuilder createDistributedLogClientBuilder(ServerSet serverSet) {
+        return DistributedLogClientBuilder.newBuilder()
+                        .name("rebalancer_tool")
+                        .clientId(ClientId$.MODULE$.apply("rebalancer_tool"))
+                        .maxRedirects(2)
+                        .serverSet(serverSet)
+                        .clientBuilder(ClientBuilder.get()
+                                .connectionTimeout(Duration.fromSeconds(2))
+                                .tcpConnectTimeout(Duration.fromSeconds(2))
+                                .requestTimeout(Duration.fromSeconds(10))
+                                .hostConnectionLimit(1)
+                                .hostConnectionCoresize(1)
+                                .keepAlive(true)
+                                .failFast(false));
+    }
+
+    /**
+     * Base Command to run balancer.
+     */
+    protected abstract static class BalancerCommand extends OptsCommand {
+
+        protected Options options = new Options();
+        protected int rebalanceWaterMark = 0;
+        protected double rebalanceTolerancePercentage = 0.0f;
+        protected int rebalanceConcurrency = 1;
+        protected Double rate = null;
+        protected Optional<RateLimiter> rateLimiter;
+
+        BalancerCommand(String name, String description) {
+            super(name, description);
+            options.addOption("rwm", "rebalance-water-mark", true, "Rebalance water mark per proxy");
+            options.addOption("rtp", "rebalance-tolerance-percentage", true,
+                "Rebalance tolerance percentage per proxy");
+            options.addOption("rc", "rebalance-concurrency", true, "Concurrency to rebalance stream distribution");
+            options.addOption("r", "rate", true, "Rebalance rate");
+        }
+
+        Optional<RateLimiter> getRateLimiter() {
+            return rateLimiter;
+        }
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (cmdline.hasOption("rwm")) {
+                this.rebalanceWaterMark = Integer.parseInt(cmdline.getOptionValue("rwm"));
+            }
+            if (cmdline.hasOption("rtp")) {
+                this.rebalanceTolerancePercentage = Double.parseDouble(cmdline.getOptionValue("rtp"));
+            }
+            if (cmdline.hasOption("rc")) {
+                this.rebalanceConcurrency = Integer.parseInt(cmdline.getOptionValue("rc"));
+            }
+            if (cmdline.hasOption("r")) {
+                this.rate = Double.parseDouble(cmdline.getOptionValue("r"));
+            }
+            checkArgument(rebalanceWaterMark >= 0,
+                    "Rebalance Water Mark should be a non-negative number");
+            checkArgument(rebalanceTolerancePercentage >= 0.0f,
+                    "Rebalance Tolerance Percentage should be a non-negative number");
+            checkArgument(rebalanceConcurrency > 0,
+                    "Rebalance Concurrency should be a positive number");
+            if (null == rate || rate <= 0.0f) {
+                rateLimiter = Optional.absent();
+            } else {
+                rateLimiter = Optional.of(RateLimiter.create(rate));
+            }
+        }
+
+        @Override
+        protected int runCmd(CommandLine cmdline) throws Exception {
+            try {
+                parseCommandLine(cmdline);
+            } catch (ParseException pe) {
+                println("ERROR: fail to parse commandline : '" + pe.getMessage() + "'");
+                printUsage();
+                return -1;
+            }
+            return executeCommand(cmdline);
+        }
+
+        protected abstract int executeCommand(CommandLine cmdline) throws Exception;
+    }
+
+    /**
+     * Command to balance streams within a cluster.
+     */
+    protected static class ClusterBalancerCommand extends BalancerCommand {
+
+        protected URI uri;
+        protected String source = null;
+
+        protected ClusterBalancerCommand() {
+            super("clusterbalancer", "Balance streams inside a cluster");
+            options.addOption("u", "uri", true, "DistributedLog URI");
+            options.addOption("sp", "source-proxy", true, "Source proxy to balance");
+        }
+
+        @Override
+        protected String getUsage() {
+            return "clusterbalancer [options]";
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("u")) {
+                throw new ParseException("No proxy serverset provided.");
+            }
+            uri = URI.create(cmdline.getOptionValue("u"));
+            if (cmdline.hasOption("sp")) {
+                String sourceProxyStr = cmdline.getOptionValue("sp");
+                try {
+                    DLSocketAddress.parseSocketAddress(sourceProxyStr);
+                } catch (IllegalArgumentException iae) {
+                    throw new ParseException("Invalid source proxy " + sourceProxyStr + " : " + iae.getMessage());
+                }
+                this.source = sourceProxyStr;
+            }
+        }
+
+        @Override
+        protected int executeCommand(CommandLine cmdline) throws Exception {
+            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
+            logger.info("Created serverset for {}", uri);
+            try {
+                DistributedLogClientBuilder clientBuilder =
+                        createDistributedLogClientBuilder(serverSet.getServerSet());
+                ClusterBalancer balancer = new ClusterBalancer(clientBuilder);
+                try {
+                    return runBalancer(clientBuilder, balancer);
+                } finally {
+                    balancer.close();
+                }
+            } finally {
+                serverSet.close();
+            }
+        }
+
+        protected int runBalancer(DistributedLogClientBuilder clientBuilder,
+                                  ClusterBalancer balancer)
+                throws Exception {
+            if (null == source) {
+                balancer.balance(
+                    rebalanceWaterMark,
+                    rebalanceTolerancePercentage,
+                    rebalanceConcurrency,
+                    getRateLimiter());
+            } else {
+                balanceFromSource(clientBuilder, balancer, source, getRateLimiter());
+            }
+            return 0;
+        }
+
+        protected void balanceFromSource(DistributedLogClientBuilder clientBuilder,
+                                         ClusterBalancer balancer,
+                                         String source,
+                                         Optional<RateLimiter> rateLimiter)
+                throws Exception {
+            InetSocketAddress sourceAddr = DLSocketAddress.parseSocketAddress(source);
+            DistributedLogClientBuilder sourceClientBuilder =
+                    DistributedLogClientBuilder.newBuilder(clientBuilder)
+                            .host(sourceAddr);
+
+            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
+                    ClientUtils.buildClient(sourceClientBuilder);
+            try {
+                Await.result(clientPair.getRight().setAcceptNewStream(false));
+                logger.info("Disable accepting new stream on proxy {}.", source);
+                balancer.balanceAll(source, rebalanceConcurrency, rateLimiter);
+            } finally {
+                clientPair.getLeft().close();
+            }
+        }
+    }
+
+    /**
+     * Command to balance streams between regions.
+     */
+    protected static class RegionBalancerCommand extends BalancerCommand {
+
+        protected URI region1;
+        protected URI region2;
+        protected String source = null;
+
+        protected RegionBalancerCommand() {
+            super("regionbalancer", "Balance streams between regions");
+            options.addOption("rs", "regions", true, "DistributedLog Region URI: uri1[,uri2]");
+            options.addOption("s", "source", true, "DistributedLog Source Region to balance");
+        }
+
+        @Override
+        protected String getUsage() {
+            return "regionbalancer [options]";
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("rs")) {
+                throw new ParseException("No regions provided.");
+            }
+            String regionsStr = cmdline.getOptionValue("rs");
+            String[] regions = regionsStr.split(",");
+            if (regions.length != 2) {
+                throw new ParseException("Invalid regions provided. Expected : serverset1[,serverset2]");
+            }
+            region1 = URI.create(regions[0]);
+            region2 = URI.create(regions[1]);
+            if (cmdline.hasOption("s")) {
+                source = cmdline.getOptionValue("s");
+            }
+        }
+
+        @Override
+        protected int executeCommand(CommandLine cmdline) throws Exception {
+            DLZkServerSet serverSet1 = DLZkServerSet.of(region1, 60000);
+            logger.info("Created serverset for {}", region1);
+            DLZkServerSet serverSet2 = DLZkServerSet.of(region2, 60000);
+            logger.info("Created serverset for {}", region2);
+            try {
+                DistributedLogClientBuilder builder1 =
+                        createDistributedLogClientBuilder(serverSet1.getServerSet());
+                Pair<DistributedLogClient, MonitorServiceClient> pair1 =
+                        ClientUtils.buildClient(builder1);
+                DistributedLogClientBuilder builder2 =
+                        createDistributedLogClientBuilder(serverSet2.getServerSet());
+                Pair<DistributedLogClient, MonitorServiceClient> pair2 =
+                        ClientUtils.buildClient(builder2);
+                try {
+                    SimpleBalancer balancer = new SimpleBalancer(
+                            BKNamespaceDriver.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
+                            BKNamespaceDriver.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
+                    try {
+                        return runBalancer(balancer);
+                    } finally {
+                        balancer.close();
+                    }
+                } finally {
+                    pair1.getLeft().close();
+                    pair2.getLeft().close();
+                }
+            } finally {
+                serverSet1.close();
+                serverSet2.close();
+            }
+        }
+
+        protected int runBalancer(SimpleBalancer balancer) throws Exception {
+            if (null == source) {
+                balancer.balance(
+                    rebalanceWaterMark,
+                    rebalanceTolerancePercentage,
+                    rebalanceConcurrency,
+                    getRateLimiter());
+            } else {
+                balancer.balanceAll(source, rebalanceConcurrency, getRateLimiter());
+            }
+            return 0;
+        }
+    }
+
+    public BalancerTool() {
+        super();
+        addCommand(new ClusterBalancerCommand());
+        addCommand(new RegionBalancerCommand());
+    }
+
+    @Override
+    protected String getName() {
+        return "balancer";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java
new file mode 100644
index 0000000..4c9e075
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.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.service.balancer;
+
+import java.util.Map;
+
+/**
+ * Utils for balancer.
+ */
+public class BalancerUtils {
+
+    /**
+     * Util function to calculate how many streams to balance for <i>nodeToRebalance</i>,
+     * based on the load distribution <i>loadDistribution</i>.
+     *
+     * @param nodeToRebalance
+     *          node to rebalance
+     * @param loadDistribution
+     *          load distribution map
+     * @param rebalanceWaterMark
+     *          if number of streams of <i>nodeToRebalance</i>
+     *          is less than <i>rebalanceWaterMark</i>, no streams will be re-balanced.
+     * @param tolerancePercentage
+     *          tolerance percentage for the balancer. if number of streams of <i>nodeToRebalance</i>
+     *          is less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
+     *          be re-balanced.
+     * @param <K>
+     * @return number of streams to rebalance
+     */
+    public static <K> int calculateNumStreamsToRebalance(K nodeToRebalance,
+                                                         Map<K, Integer> loadDistribution,
+                                                         int rebalanceWaterMark,
+                                                         double tolerancePercentage) {
+        Integer myLoad = loadDistribution.get(nodeToRebalance);
+        if (null == myLoad || myLoad <= rebalanceWaterMark) {
+            return 0;
+        }
+
+        long totalLoad = 0L;
+        int numNodes = loadDistribution.size();
+
+        for (Map.Entry<K, Integer> entry : loadDistribution.entrySet()) {
+            if (null == entry.getKey() || null == entry.getValue()) {
+                continue;
+            }
+            totalLoad += entry.getValue();
+        }
+
+        double averageLoad = ((double) totalLoad) / numNodes;
+        long permissibleLoad =
+                Math.max(1L, (long) Math.ceil(averageLoad + averageLoad * tolerancePercentage / 100.0f));
+
+        if (myLoad <= permissibleLoad) {
+            return 0;
+        }
+
+        return Math.max(0, myLoad - (int) Math.ceil(averageLoad));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
new file mode 100644
index 0000000..5add339
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
@@ -0,0 +1,378 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.ClientUtils;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import com.twitter.util.Await;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.io.Serializable;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A balancer balances ownerships with a cluster of targets.
+ */
+public class ClusterBalancer implements Balancer {
+
+    private static final Logger logger = LoggerFactory.getLogger(ClusterBalancer.class);
+
+    /**
+     * Represent a single host. Ordered by number of streams in desc order.
+     */
+    static class Host {
+
+        final SocketAddress address;
+        final Set<String> streams;
+        final DistributedLogClientBuilder clientBuilder;
+        DistributedLogClient client = null;
+        MonitorServiceClient monitor = null;
+
+        Host(SocketAddress address, Set<String> streams,
+             DistributedLogClientBuilder clientBuilder) {
+            this.address = address;
+            this.streams = streams;
+            this.clientBuilder = clientBuilder;
+        }
+
+        private void initializeClientsIfNeeded() {
+            if (null == client) {
+                Pair<DistributedLogClient, MonitorServiceClient> clientPair =
+                        createDistributedLogClient(address, clientBuilder);
+                client = clientPair.getLeft();
+                monitor = clientPair.getRight();
+            }
+        }
+
+        synchronized DistributedLogClient getClient() {
+            initializeClientsIfNeeded();
+            return client;
+        }
+
+        synchronized MonitorServiceClient getMonitor() {
+            initializeClientsIfNeeded();
+            return monitor;
+        }
+
+        synchronized void close() {
+            if (null != client) {
+                client.close();
+            }
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append("Host(").append(address).append(")");
+            return sb.toString();
+        }
+    }
+
+    static class HostComparator implements Comparator<Host>, Serializable {
+        private static final long serialVersionUID = 7984973796525102538L;
+
+        @Override
+        public int compare(Host h1, Host h2) {
+            return h2.streams.size() - h1.streams.size();
+        }
+    }
+
+    protected final DistributedLogClientBuilder clientBuilder;
+    protected final DistributedLogClient client;
+    protected final MonitorServiceClient monitor;
+
+    public ClusterBalancer(DistributedLogClientBuilder clientBuilder) {
+        this(clientBuilder, ClientUtils.buildClient(clientBuilder));
+    }
+
+    ClusterBalancer(DistributedLogClientBuilder clientBuilder,
+                    Pair<DistributedLogClient, MonitorServiceClient> clientPair) {
+        this.clientBuilder = clientBuilder;
+        this.client = clientPair.getLeft();
+        this.monitor = clientPair.getRight();
+    }
+
+    /**
+     * Build a new distributedlog client to a single host <i>host</i>.
+     *
+     * @param host
+     *          host to access
+     * @return distributedlog clients
+     */
+    static Pair<DistributedLogClient, MonitorServiceClient> createDistributedLogClient(
+            SocketAddress host, DistributedLogClientBuilder clientBuilder) {
+        DistributedLogClientBuilder newBuilder =
+                DistributedLogClientBuilder.newBuilder(clientBuilder).host(host);
+        return ClientUtils.buildClient(newBuilder);
+    }
+
+    @Override
+    public void balanceAll(String source,
+                           int rebalanceConcurrency, /* unused */
+                           Optional<RateLimiter> rebalanceRateLimiter) {
+        balance(0, 0.0f, rebalanceConcurrency, Optional.of(source), rebalanceRateLimiter);
+    }
+
+    @Override
+    public void balance(int rebalanceWaterMark,
+                        double rebalanceTolerancePercentage,
+                        int rebalanceConcurrency, /* unused */
+                        Optional<RateLimiter> rebalanceRateLimiter) {
+        Optional<String> source = Optional.absent();
+        balance(rebalanceWaterMark, rebalanceTolerancePercentage, rebalanceConcurrency, source, rebalanceRateLimiter);
+    }
+
+    public void balance(int rebalanceWaterMark,
+                        double rebalanceTolerancePercentage,
+                        int rebalanceConcurrency,
+                        Optional<String> source,
+                        Optional<RateLimiter> rebalanceRateLimiter) {
+        Map<SocketAddress, Set<String>> distribution = monitor.getStreamOwnershipDistribution();
+        if (distribution.size() <= 1) {
+            return;
+        }
+        SocketAddress sourceAddr = null;
+        if (source.isPresent()) {
+            sourceAddr = DLSocketAddress.parseSocketAddress(source.get());
+            logger.info("Balancer source is {}", sourceAddr);
+            if (!distribution.containsKey(sourceAddr)) {
+                return;
+            }
+        }
+        // Get the list of hosts ordered by number of streams in DESC order
+        List<Host> hosts = new ArrayList<Host>(distribution.size());
+        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
+            Host host = new Host(entry.getKey(), entry.getValue(), clientBuilder);
+            hosts.add(host);
+        }
+        Collections.sort(hosts, new HostComparator());
+        try {
+
+            // find the host to move streams from.
+            int hostIdxMoveFrom = -1;
+            if (null != sourceAddr) {
+                for (Host host : hosts) {
+                    ++hostIdxMoveFrom;
+                    if (sourceAddr.equals(host.address)) {
+                        break;
+                    }
+                }
+            }
+
+            // compute the average load.
+            int totalStream = 0;
+            for (Host host : hosts) {
+                totalStream += host.streams.size();
+            }
+            double averageLoad;
+            if (hostIdxMoveFrom >= 0) {
+                averageLoad = ((double) totalStream / (hosts.size() - 1));
+            } else {
+                averageLoad = ((double) totalStream / hosts.size());
+            }
+
+            int moveFromLowWaterMark;
+            int moveToHighWaterMark =
+                Math.max(1, (int) (averageLoad + averageLoad * rebalanceTolerancePercentage / 100.0f));
+
+            if (hostIdxMoveFrom >= 0) {
+                moveFromLowWaterMark = Math.max(0, rebalanceWaterMark);
+                moveStreams(
+                        hosts,
+                        new AtomicInteger(hostIdxMoveFrom), moveFromLowWaterMark,
+                        new AtomicInteger(hosts.size() - 1), moveToHighWaterMark,
+                        rebalanceRateLimiter);
+                moveRemainingStreamsFromSource(hosts.get(hostIdxMoveFrom), hosts, rebalanceRateLimiter);
+            } else {
+                moveFromLowWaterMark = Math.max((int) Math.ceil(averageLoad), rebalanceWaterMark);
+                AtomicInteger moveFrom = new AtomicInteger(0);
+                AtomicInteger moveTo = new AtomicInteger(hosts.size() - 1);
+                while (moveFrom.get() < moveTo.get()) {
+                    moveStreams(hosts, moveFrom, moveFromLowWaterMark,
+                        moveTo, moveToHighWaterMark, rebalanceRateLimiter);
+                    moveFrom.incrementAndGet();
+                }
+            }
+        } finally {
+            for (Host host : hosts) {
+                host.close();
+            }
+        }
+    }
+
+    void moveStreams(List<Host> hosts,
+                     AtomicInteger hostIdxMoveFrom,
+                     int moveFromLowWaterMark,
+                     AtomicInteger hostIdxMoveTo,
+                     int moveToHighWaterMark,
+                     Optional<RateLimiter> rateLimiter) {
+        if (hostIdxMoveFrom.get() < 0 || hostIdxMoveFrom.get() >= hosts.size()
+                || hostIdxMoveTo.get() < 0 || hostIdxMoveTo.get() >= hosts.size()
+                || hostIdxMoveFrom.get() >= hostIdxMoveTo.get()) {
+            return;
+        }
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Moving streams : hosts = {}, from = {}, to = {} :"
+                + " from_low_water_mark = {}, to_high_water_mark = {}",
+                new Object[] {
+                    hosts,
+                    hostIdxMoveFrom.get(),
+                    hostIdxMoveTo.get(),
+                    moveFromLowWaterMark,
+                    moveToHighWaterMark });
+        }
+
+        Host hostMoveFrom = hosts.get(hostIdxMoveFrom.get());
+        int numStreamsOnFromHost = hostMoveFrom.streams.size();
+        if (numStreamsOnFromHost <= moveFromLowWaterMark) {
+            // do nothing
+            return;
+        }
+
+        int numStreamsToMove = numStreamsOnFromHost - moveFromLowWaterMark;
+        LinkedList<String> streamsToMove = new LinkedList<String>(hostMoveFrom.streams);
+        Collections.shuffle(streamsToMove);
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Try to move {} streams from host {} : streams = {}",
+                         new Object[] { numStreamsToMove, hostMoveFrom.address, streamsToMove });
+        }
+
+        while (numStreamsToMove-- > 0 && !streamsToMove.isEmpty()) {
+            if (rateLimiter.isPresent()) {
+                rateLimiter.get().acquire();
+            }
+
+            // pick a host to move
+            Host hostMoveTo = hosts.get(hostIdxMoveTo.get());
+            while (hostMoveTo.streams.size() >= moveToHighWaterMark) {
+                int hostIdx = hostIdxMoveTo.decrementAndGet();
+                logger.info("move to host : {}, from {}", hostIdx, hostIdxMoveFrom.get());
+                if (hostIdx <= hostIdxMoveFrom.get()) {
+                    return;
+                } else {
+                    hostMoveTo = hosts.get(hostIdx);
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("Target host to move moved to host {} @ {}",
+                                hostIdx, hostMoveTo);
+                    }
+                }
+            }
+
+            // pick a stream
+            String stream = streamsToMove.remove();
+
+            // move the stream
+            if (moveStream(stream, hostMoveFrom, hostMoveTo)) {
+                hostMoveFrom.streams.remove(stream);
+                hostMoveTo.streams.add(stream);
+            }
+        }
+
+    }
+
+    void moveRemainingStreamsFromSource(Host source,
+                                        List<Host> hosts,
+                                        Optional<RateLimiter> rateLimiter) {
+        LinkedList<String> streamsToMove = new LinkedList<String>(source.streams);
+        Collections.shuffle(streamsToMove);
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Try to move remaining streams from {} : {}", source, streamsToMove);
+        }
+
+        int hostIdx = hosts.size() - 1;
+
+        while (!streamsToMove.isEmpty()) {
+            if (rateLimiter.isPresent()) {
+                rateLimiter.get().acquire();
+            }
+
+            Host target = hosts.get(hostIdx);
+            if (!target.address.equals(source.address)) {
+                String stream = streamsToMove.remove();
+                // move the stream
+                if (moveStream(stream, source, target)) {
+                    source.streams.remove(stream);
+                    target.streams.add(stream);
+                }
+            }
+            --hostIdx;
+            if (hostIdx < 0) {
+                hostIdx = hosts.size() - 1;
+            }
+        }
+    }
+
+    private boolean moveStream(String stream, Host from, Host to) {
+        try {
+            doMoveStream(stream, from, to);
+            return true;
+        } catch (Exception e) {
+            return false;
+        }
+    }
+
+    private void doMoveStream(final String stream, final Host from, final Host to) throws Exception {
+        logger.info("Moving stream {} from {} to {}.",
+                    new Object[] { stream, from.address, to.address });
+        Await.result(from.getClient().release(stream).flatMap(new Function<Void, Future<Void>>() {
+            @Override
+            public Future<Void> apply(Void result) {
+                logger.info("Released stream {} from {}.", stream, from.address);
+                return to.getMonitor().check(stream).addEventListener(new FutureEventListener<Void>() {
+
+                    @Override
+                    public void onSuccess(Void value) {
+                        logger.info("Moved stream {} from {} to {}.",
+                                    new Object[] { stream, from.address, to.address });
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        logger.info("Failed to move stream {} from {} to {} : ",
+                                    new Object[] { stream, from.address, to.address, cause });
+                    }
+                });
+            }
+        }));
+    }
+
+    @Override
+    public void close() {
+        client.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java
new file mode 100644
index 0000000..6a43179
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.balancer;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.Serializable;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * A stream chooser based on number of streams.
+ */
+class CountBasedStreamChooser implements StreamChooser, Serializable,
+        Comparator<Pair<SocketAddress, LinkedList<String>>> {
+
+    private static final long serialVersionUID = 4664153397369979203L;
+
+    final List<Pair<SocketAddress, LinkedList<String>>> streamsDistribution;
+
+    // pivot index in the list of hosts. the chooser will remove streams from the hosts before
+    // pivot, which will reduce their stream counts to make them equal to the stream count of the pivot.
+    int pivot;
+    int pivotCount;
+
+    // next index in the list of hosts to choose stream from.
+    int next;
+
+    CountBasedStreamChooser(Map<SocketAddress, Set<String>> streams) {
+        checkArgument(streams.size() > 0, "Only support no-empty streams distribution");
+        streamsDistribution = new ArrayList<Pair<SocketAddress, LinkedList<String>>>(streams.size());
+        for (Map.Entry<SocketAddress, Set<String>> entry : streams.entrySet()) {
+            LinkedList<String> randomizedStreams = new LinkedList<String>(entry.getValue());
+            Collections.shuffle(randomizedStreams);
+            streamsDistribution.add(Pair.of(entry.getKey(), randomizedStreams));
+        }
+        // sort the hosts by the number of streams in descending order
+        Collections.sort(streamsDistribution, this);
+        pivot = 0;
+        pivotCount = streamsDistribution.get(0).getValue().size();
+        findNextPivot();
+        next = 0;
+    }
+
+    private void findNextPivot() {
+        int prevPivotCount = pivotCount;
+        while (++pivot < streamsDistribution.size()) {
+            pivotCount = streamsDistribution.get(pivot).getValue().size();
+            if (pivotCount < prevPivotCount) {
+                return;
+            }
+        }
+        pivot = streamsDistribution.size();
+        pivotCount = 0;
+    }
+
+    @Override
+    public synchronized String choose() {
+        // reach the pivot
+        if (next == pivot) {
+            if (streamsDistribution.get(next - 1).getRight().size() > pivotCount) {
+                next = 0;
+            } else if (pivotCount == 0) { // the streams are empty now
+                return null;
+            } else {
+                findNextPivot();
+                next = 0;
+            }
+        }
+
+        // get stream count that next host to choose from
+        LinkedList<String> nextStreams = streamsDistribution.get(next).getRight();
+        if (nextStreams.size() == 0) {
+            return null;
+        }
+
+        String chosenStream = nextStreams.remove();
+        ++next;
+        return chosenStream;
+    }
+
+    @Override
+    public int compare(Pair<SocketAddress, LinkedList<String>> o1,
+                       Pair<SocketAddress, LinkedList<String>> o2) {
+        return o2.getValue().size() - o1.getValue().size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java
new file mode 100644
index 0000000..4aefc5e
--- /dev/null
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.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.service.balancer;
+
+/**
+ * A stream chooser that can only choose limited number of streams.
+ */
+public class LimitedStreamChooser implements StreamChooser {
+
+  /**
+   * Create a limited stream chooser by {@code limit}.
+   *
+   * @param underlying the underlying stream chooser.
+   * @param limit the limit of number of streams to choose.
+   * @return the limited stream chooser.
+   */
+    public static LimitedStreamChooser of(StreamChooser underlying, int limit) {
+        return new LimitedStreamChooser(underlying, limit);
+    }
+
+    final StreamChooser underlying;
+    int limit;
+
+    LimitedStreamChooser(StreamChooser underlying, int limit) {
+        this.underlying = underlying;
+        this.limit = limit;
+    }
+
+    @Override
+    public synchronized String choose() {
+        if (limit <= 0) {
+            return null;
+        }
+        String s = underlying.choose();
+        if (s == null) {
+            limit = 0;
+            return null;
+        }
+        --limit;
+        return s;
+    }
+}



[30/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
DL-205: Remove StatusCode dependency on DLException

- Remove StatusCode from exceptions. Use integer as exception codes.
- Also re-organize the modules:

- [ ] distributedlog-protocol (for core structures) and distributedlog-core (for core library).
- [ ] proxy: distributedlog-proxy-protocol (new module for thrift generated protocol), distributedlog-proxy-client (proxy client) and distributedlog-proxy-server (proxy server)
- [ ] benchmark & tutorials.

Author: Sijie Guo <si...@apache.org>

Reviewers: Jia Zhai <None>, Leigh Stewart <ls...@apache.org>

Closes #131 from sijie/new_layout


Project: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/commit/c44e0278
Tree: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/tree/c44e0278
Diff: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/diff/c44e0278

Branch: refs/heads/master
Commit: c44e0278ececde1942d5c43b06c7a12d375974a8
Parents: 52c0eef
Author: Sijie Guo <si...@apache.org>
Authored: Mon Jun 12 08:45:01 2017 -0700
Committer: Sijie Guo <si...@apache.org>
Committed: Mon Jun 12 08:45:01 2017 -0700

----------------------------------------------------------------------
 distributedlog-benchmark/pom.xml                |    2 +-
 .../distributedlog/benchmark/WriterWorker.java  |    3 +-
 distributedlog-client/pom.xml                   |  172 ---
 .../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      |   23 -
 .../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 -
 .../src/test/resources/log4j.properties         |   51 -
 .../distributedlog/exceptions/ZKException.java  |    1 -
 distributedlog-protocol/pom.xml                 |   45 +-
 .../exceptions/AlreadyClosedException.java      |    2 -
 .../AlreadyTruncatedTransactionException.java   |    2 -
 .../exceptions/BKTransmitException.java         |    2 -
 .../exceptions/ChecksumFailedException.java     |    2 -
 .../exceptions/DLClientClosedException.java     |    2 -
 .../distributedlog/exceptions/DLException.java  |   35 +-
 .../exceptions/DLIllegalStateException.java     |    2 -
 .../exceptions/DLInterruptedException.java      |    2 -
 .../exceptions/EndOfLogSegmentException.java    |    2 -
 .../exceptions/EndOfStreamException.java        |    2 -
 .../exceptions/FlushException.java              |    2 -
 .../exceptions/InternalServerException.java     |    2 -
 .../InvalidEnvelopedEntryException.java         |    2 -
 .../exceptions/InvalidStreamNameException.java  |    2 -
 .../exceptions/LockingException.java            |    6 +-
 .../exceptions/LogEmptyException.java           |    2 -
 .../exceptions/LogExistsException.java          |    2 -
 .../exceptions/LogNotFoundException.java        |    2 -
 .../exceptions/LogReadException.java            |    2 -
 .../exceptions/LogRecordTooLongException.java   |    2 -
 .../LogSegmentIsTruncatedException.java         |    2 -
 .../exceptions/LogSegmentNotFoundException.java |    2 -
 .../exceptions/MetadataException.java           |    2 -
 .../exceptions/NotYetImplementedException.java  |    2 -
 .../exceptions/OverCapacityException.java       |    4 +-
 .../OwnershipAcquireFailedException.java        |    2 -
 .../exceptions/ReadCancelledException.java      |    2 -
 .../exceptions/RegionUnavailableException.java  |    2 -
 .../exceptions/RequestDeniedException.java      |    2 -
 .../exceptions/RetryableReadException.java      |    2 -
 .../exceptions/ServiceUnavailableException.java |    2 -
 .../distributedlog/exceptions/StatusCode.java   |  122 ++
 .../exceptions/StreamNotReadyException.java     |    2 -
 .../exceptions/StreamUnavailableException.java  |    2 -
 .../exceptions/TooManyStreamsException.java     |    2 -
 .../TransactionIdOutOfOrderException.java       |    2 -
 .../exceptions/UnexpectedException.java         |    2 -
 .../UnsupportedMetadataVersionException.java    |    2 -
 .../exceptions/WriteCancelledException.java     |    2 -
 .../exceptions/WriteException.java              |    2 -
 .../distributedlog/util/ProtocolUtils.java      |   79 --
 .../src/main/resources/findbugsExclude.xml      |    4 -
 .../src/main/thrift/service.thrift              |  203 ---
 distributedlog-proxy-client/pom.xml             |  172 +++
 .../distributedlog/client/ClientConfig.java     |  187 +++
 .../client/DistributedLogClientImpl.java        | 1199 +++++++++++++++++
 .../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      |   23 +
 .../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 +
 .../src/test/resources/log4j.properties         |   51 +
 distributedlog-proxy-protocol/pom.xml           |  130 ++
 .../protocol/util/ProtocolUtils.java            |  104 ++
 .../protocol/util/package-info.java             |   21 +
 .../src/main/resources/findbugsExclude.xml      |   23 +
 .../src/main/thrift/service.thrift              |  203 +++
 distributedlog-proxy-server/bin/bundle          |   28 +
 distributedlog-proxy-server/bin/common.sh       |  124 ++
 distributedlog-proxy-server/bin/dlog            |   88 ++
 distributedlog-proxy-server/bin/dlog-daemon.sh  |  200 +++
 distributedlog-proxy-server/bin/dlog-env.sh     |   35 +
 distributedlog-proxy-server/bin/dlog-start.sh   |   34 +
 .../bin/zk-server-start.sh                      |   44 +
 .../conf/bookie.conf.template                   |  183 +++
 .../conf/distributedlog.conf                    |  125 ++
 distributedlog-proxy-server/conf/dlogenv.sh     |   75 ++
 .../conf/log4j.properties                       |   60 +
 .../conf/write_proxy.conf                       |  143 +++
 .../conf/zookeeper.conf.template                |   82 ++
 distributedlog-proxy-server/pom.xml             |  275 ++++
 .../stats/CodahaleMetricsServletProvider.java   |   63 +
 .../HealthCheckServletContextListener.java      |   38 +
 .../stats/MetricsServletContextListener.java    |   38 +
 .../bookkeeper/stats/ServletReporter.java       |   62 +
 .../apache/bookkeeper/stats/package-info.java   |   21 +
 .../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             |  102 ++
 .../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              |  926 ++++++++++++++
 .../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 +
 .../main/resources/config/server_decider.conf   |   31 +
 .../main/resources/config/server_decider.yml    |   44 +
 .../src/main/resources/findbugsExclude.xml      |   39 +
 .../src/main/thrift/metadata.thrift             |   29 +
 .../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 +
 .../src/test/resources/log4j.properties         |   51 +
 distributedlog-service/bin/bundle               |   28 -
 distributedlog-service/bin/common.sh            |  124 --
 distributedlog-service/bin/dlog                 |   88 --
 distributedlog-service/bin/dlog-daemon.sh       |  200 ---
 distributedlog-service/bin/dlog-env.sh          |   35 -
 distributedlog-service/bin/dlog-start.sh        |   34 -
 distributedlog-service/bin/zk-server-start.sh   |   44 -
 .../conf/bookie.conf.template                   |  183 ---
 distributedlog-service/conf/distributedlog.conf |  125 --
 distributedlog-service/conf/dlogenv.sh          |   75 --
 distributedlog-service/conf/log4j.properties    |   60 -
 distributedlog-service/conf/write_proxy.conf    |  143 ---
 .../conf/zookeeper.conf.template                |   82 --
 distributedlog-service/pom.xml                  |  270 ----
 .../stats/CodahaleMetricsServletProvider.java   |   63 -
 .../HealthCheckServletContextListener.java      |   38 -
 .../stats/MetricsServletContextListener.java    |   38 -
 .../bookkeeper/stats/ServletReporter.java       |   62 -
 .../apache/bookkeeper/stats/package-info.java   |   21 -
 .../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             |  102 --
 .../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 -
 .../main/resources/config/server_decider.conf   |   31 -
 .../main/resources/config/server_decider.yml    |   44 -
 .../src/main/resources/findbugsExclude.xml      |   39 -
 .../src/main/thrift/metadata.thrift             |   29 -
 .../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 -
 .../src/test/resources/log4j.properties         |   51 -
 .../distributedlog-basic/pom.xml                |    2 +-
 .../distributedlog-mapreduce/pom.xml            |    2 +-
 .../distributedlog-messaging/pom.xml            |    2 +-
 pom.xml                                         |    7 +-
 438 files changed, 26309 insertions(+), 26118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-benchmark/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/pom.xml b/distributedlog-benchmark/pom.xml
index 1184907..26651ad 100644
--- a/distributedlog-benchmark/pom.xml
+++ b/distributedlog-benchmark/pom.xml
@@ -27,7 +27,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-client</artifactId>
+      <artifactId>distributedlog-proxy-client</artifactId>
       <version>${project.parent.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
index 9e96765..fa96dfb 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
@@ -28,6 +28,7 @@ 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.thrift.service.StatusCode;
 import org.apache.distributedlog.util.SchedulerUtils;
 import com.twitter.finagle.builder.ClientBuilder;
 import com.twitter.finagle.stats.StatsReceiver;
@@ -278,7 +279,7 @@ public class WriterWorker implements Worker {
                 exceptionsLogger.getCounter(cause.getClass().getName()).inc();
                 if (cause instanceof DLException) {
                     DLException dle = (DLException) cause;
-                    dlErrorCodeLogger.getCounter(dle.getCode().toString()).inc();
+                    dlErrorCodeLogger.getCounter(StatusCode.findByValue(dle.getCode()).toString()).inc();
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-client/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-client/pom.xml b/distributedlog-client/pom.xml
deleted file mode 100644
index 5f3aa16..0000000
--- a/distributedlog-client/pom.xml
+++ /dev/null
@@ -1,172 +0,0 @@
-<?xml version="1.0"?>
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.distributedlog</groupId>
-    <artifactId>distributedlog</artifactId>
-    <version>0.5.0-incubating-SNAPSHOT</version>
-  </parent>
-  <artifactId>distributedlog-client</artifactId>
-  <name>Apache DistributedLog :: Proxy Client</name>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-protocol</artifactId>
-      <version>${project.parent.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>finagle-core_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>finagle-thriftmux_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>finagle-serversets_2.11</artifactId>
-      <version>${finagle.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-jdk14</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <version>${guava.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-lang3</artifactId>
-      <version>${commons-lang3.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${junit.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-      <version>${slf4j.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <version>${slf4j.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <version>${mockito.version}</version>
-      <scope>test</scope>
-    </dependency> 
-    <dependency>
-      <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-protocol</artifactId>
-      <version>${project.parent.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-        <configuration>
-          <excludeFilterFile>${basedir}/src/main/resources/findbugsExclude.xml</excludeFilterFile>
-        </configuration>
-      </plugin>
-      <plugin>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <version>${maven-compiler-plugin.version}</version>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <version>${maven-jar-plugin.version}</version>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <version>${maven-surefire-plugin.version}</version>
-        <configuration>
-          <redirectTestOutputToFile>true</redirectTestOutputToFile>
-          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
-          <forkMode>always</forkMode>
-          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.distributedlog.TimedOutTestsListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-        <version>${maven-checkstyle-plugin.version}</version>
-        <dependencies>
-          <dependency>
-            <groupId>com.puppycrawl.tools</groupId>
-            <artifactId>checkstyle</artifactId>
-            <version>${puppycrawl.checkstyle.version}</version>
-          </dependency>
-          <dependency>
-            <groupId>org.apache.distributedlog</groupId>
-            <artifactId>distributedlog-build-tools</artifactId>
-            <version>${project.version}</version>
-          </dependency>
-        </dependencies>
-        <configuration>
-          <configLocation>distributedlog/checkstyle.xml</configLocation>
-          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
-          <consoleOutput>true</consoleOutput>
-          <failOnViolation>true</failOnViolation>
-          <includeResources>false</includeResources>
-          <includeTestSourceDirectory>true</includeTestSourceDirectory>
-        </configuration>
-        <executions>
-          <execution>
-            <phase>test-compile</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index 57e2b5a..0000000
--- a/distributedlog-client/src/main/java/org/apache/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 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;
-    }
-}



[25/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index f44cddd..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 59665b9..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index 151663e..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/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
deleted file mode 100644
index d2d61a9..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index ab0cb58..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/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
deleted file mode 100644
index d2df9a5..0000000
--- a/distributedlog-client/src/test/java/org/apache/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 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/c44e0278/distributedlog-client/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/resources/log4j.properties b/distributedlog-client/src/test/resources/log4j.properties
deleted file mode 100644
index 3e51059..0000000
--- a/distributedlog-client/src/test/resources/log4j.properties
+++ /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.
-# */
-
-#
-# DisributedLog Logging Configuration
-#
-
-# Example with rolling log file
-log4j.rootLogger=INFO, CONSOLE
-
-#disable zookeeper logging
-log4j.logger.org.apache.zookeeper=OFF
-#Set the bookkeeper level to warning
-log4j.logger.org.apache.bookkeeper=INFO
-
-log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
-log4j.appender.CONSOLE.Threshold=INFO
-log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
-log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-# Add ROLLINGFILE to rootLogger to get log file output
-#    Log DEBUG level and above messages to a log file
-#log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
-#log4j.appender.ROLLINGFILE.Threshold=INFO
-#log4j.appender.ROLLINGFILE.File=distributedlog.log
-#log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
-#log4j.appender.ROLLINGFILE.DatePattern='.'yyyy-MM-dd-HH-mm
-#log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
-
-log4j.appender.R=org.apache.log4j.RollingFileAppender
-log4j.appender.R.Threshold=TRACE
-log4j.appender.R.File=target/error.log
-log4j.appender.R.MaxFileSize=200MB
-log4j.appender.R.MaxBackupIndex=7
-log4j.appender.R.layout=org.apache.log4j.PatternLayout
-log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
index b9e9d12..5eab707 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
@@ -17,7 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
 

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/pom.xml b/distributedlog-protocol/pom.xml
index 8a9cb22..a483444 100644
--- a/distributedlog-protocol/pom.xml
+++ b/distributedlog-protocol/pom.xml
@@ -37,26 +37,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
-      <version>${libthrift.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>scrooge-core_2.11</artifactId>
-      <version>${scrooge.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>finagle-core_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>finagle-thrift_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
       <version>${commons-lang.version}</version>
@@ -67,6 +47,11 @@
       <version>${commons-codec.version}</version>
     </dependency>
     <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>finagle-core_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <version>${slf4j.version}</version>
@@ -86,26 +71,6 @@
   <build>
     <plugins>
       <plugin>
-        <groupId>com.twitter</groupId>
-        <artifactId>scrooge-maven-plugin</artifactId>
-        <version>${scrooge-maven-plugin.version}</version>
-        <configuration>
-          <language>java</language>
-          <thriftOpts>
-            <thriftOpt>--finagle</thriftOpt>
-          </thriftOpts>
-        </configuration>
-        <executions>
-          <execution>
-            <id>thrift-sources</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>compile</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <version>${maven-jar-plugin.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java
index ecb9180..258ff05 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Thrown when any distributedlog resources have already been closed.
  *

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
index 9f8d5bc..29903ac 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Thrown when the transaction Id specified in the API is in the range that has already been truncated.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java
index 7d0a5b3..33f4a04 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Thrown when the send to bookkeeper fails.
  *

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java
index 36c7bd9..4210d66 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception thrown when checksum failures occurred.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java
index a5e4faa..bb7c2db 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exceptions thrown when a distributedlog client is closed.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java
index e974d46..928895d 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.thrift.service.StatusCode;
 import java.io.IOException;
 
 /**
@@ -26,24 +24,24 @@ import java.io.IOException;
  */
 public class DLException extends IOException {
     private static final long serialVersionUID = -4485775468586114393L;
-    protected final StatusCode code;
+    protected final int code;
 
-    protected DLException(StatusCode code) {
+    public DLException(int code) {
         super();
         this.code = code;
     }
 
-    protected DLException(StatusCode code, String msg) {
+    public DLException(int code, String msg) {
         super(msg);
         this.code = code;
     }
 
-    protected DLException(StatusCode code, Throwable t) {
+    public DLException(int code, Throwable t) {
         super(t);
         this.code = code;
     }
 
-    protected DLException(StatusCode code, String msg, Throwable t) {
+    public DLException(int code, String msg, Throwable t) {
         super(msg, t);
         this.code = code;
     }
@@ -53,29 +51,8 @@ public class DLException extends IOException {
      *
      * @return status code representing the exception.
      */
-    public StatusCode getCode() {
+    public int getCode() {
         return code;
     }
 
-    public static DLException of(ResponseHeader response) {
-        String errMsg;
-        switch (response.getCode()) {
-            case FOUND:
-                if (response.isSetErrMsg()) {
-                    errMsg = response.getErrMsg();
-                } else {
-                    errMsg = "Request is redirected to " + response.getLocation();
-                }
-                return new OwnershipAcquireFailedException(errMsg, response.getLocation());
-            case SUCCESS:
-                throw new IllegalArgumentException("Can't instantiate an exception for success response.");
-            default:
-                if (response.isSetErrMsg()) {
-                    errMsg = response.getErrMsg();
-                } else {
-                    errMsg = response.getCode().name();
-                }
-                return new DLException(response.getCode(), errMsg);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java
index 7aa8c39..8dc4116 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Signals that a method has been invoked at an illegal or inappropriate time.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java
index 6eaf8ac..7b4d5be 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * An interrupted exception wrapper indicates dl operations are interrupted.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java
index 356dde8..5c12995 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception thrown when reach end of the log segment.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java
index 4b8a931..cc2c79f 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception thrown when a reader reaches end of a sealed log stream.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java
index 7225970..b471778 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception indicates that errors occurred on flushing data.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java
index 9032c2b..ff6338a 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception indicates that there is an internal error at distributedlog service side.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java
index d46d14e..b52af61 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception thrown when encounter invalid enveloped entry.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java
index 29e2663..ca6f4db 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when encountered invalid log stream.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.java
index 63f6b0f..55bd88f 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception when a log writer attempts to acquire a lock to write data to the stream.
  */
@@ -34,11 +32,11 @@ public class LockingException extends DLException {
         this(StatusCode.LOCKING_EXCEPTION, lockPath, message, cause);
     }
 
-    protected LockingException(StatusCode code, String lockPath, String message) {
+    protected LockingException(int code, String lockPath, String message) {
         super(code, String.format("LockPath - %s: %s", lockPath, message));
     }
 
-    protected LockingException(StatusCode code, String lockPath, String message, Throwable cause) {
+    protected LockingException(int code, String lockPath, String message, Throwable cause) {
         super(code, String.format("LockPath - %s: %s", lockPath, message), cause);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java
index ba3545d..0326a9a 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exceptions are thrown when attempt to read a log stream that doesn't have any records.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java
index 3ecb80f..cb2bc65 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception thrown on creating a log stream but the log stream already exists.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java
index 3795c5a..9ac14ba 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when a reader attempts to read a log stream that doesn't exist.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java
index 91cf5c9..d2ca399 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Thrown when there's a failure to read an edit log op from disk when loading
  * edits.

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java
index 2cb9085..dd1d66d 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when attempting to write a record whose size is too larger.
  *

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java
index ac2ebda..3364940 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when reading data from a truncated log segment.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java
index 5245cdc..7fad552 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception on log segment not found.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java
index 6dba778..559972b 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when encountering metadata errors.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java
index eb709de..75eaa52 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when a method is not implemented yet.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java
index 6dc4767..4247874 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when the system is over capacity.
  *
@@ -33,7 +31,7 @@ public class OverCapacityException extends DLException {
         super(StatusCode.OVER_CAPACITY, message);
     }
 
-    public OverCapacityException(StatusCode code, String message) {
+    public OverCapacityException(int code, String message) {
         super(code, message);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java
index af85e79..b077f5e 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when a log writer attempt to acquire a lock.
  *

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java
index ac0508a..b179ce6 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Signals that a read request has been cancelled.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java
index 7a4b225..745ae6b 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception indicates that the service is not available in one region.
  *

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java
index ab730f3..e26f6db 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Signals that a request has been denied at the server.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java
index 10a1e53..1ed506f 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Signals that a read request can be retried..
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java
index ca266c2..5ab7f03 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception indicates that the service is unavailable at the server side.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StatusCode.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StatusCode.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StatusCode.java
new file mode 100644
index 0000000..6dbb931
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StatusCode.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.exceptions;
+
+/**
+ * Exception Status Code.
+ */
+public interface StatusCode {
+
+    /* 2xx: action requested by the client was received, understood, accepted and processed successfully. */
+
+    /* standard response for successful requests. */
+    int SUCCESS = 200;
+
+    /* 3xx: client must take additional action to complete the request. */
+
+    /* client closed. */
+    int CLIENT_CLOSED = 301;
+    /* found the stream in a different server, a redirection is required by client. */
+    int FOUND = 302;
+
+    /* 4xx: client seems to have erred. */
+
+    /* request is denied for some reason */
+    int REQUEST_DENIED = 403;
+    /* request record too large */
+    int TOO_LARGE_RECORD = 413;
+
+    /* 5xx: server failed to fulfill an apparently valid request. */
+
+    /* Generic error message, given when no more specific message is suitable. */
+    int INTERNAL_SERVER_ERROR = 500;
+    /* Not implemented */
+    int NOT_IMPLEMENTED = 501;
+    /* Already Closed Exception */
+    int ALREADY_CLOSED = 502;
+    /* Service is currently unavailable (because it is overloaded or down for maintenance). */
+    int SERVICE_UNAVAILABLE = 503;
+    /* Locking exception */
+    int LOCKING_EXCEPTION = 504;
+    /* ZooKeeper Errors */
+    int ZOOKEEPER_ERROR = 505;
+    /* Metadata exception */
+    int METADATA_EXCEPTION = 506;
+    /* BK Transmit Error */
+    int BK_TRANSMIT_ERROR = 507;
+    /* Flush timeout */
+    int FLUSH_TIMEOUT = 508;
+    /* Log empty */
+    int LOG_EMPTY = 509;
+    /* Log not found */
+    int LOG_NOT_FOUND = 510;
+    /* Truncated Transactions */
+    int TRUNCATED_TRANSACTION = 511;
+    /* End of Stream */
+    int END_OF_STREAM = 512;
+    /* Transaction Id Out of Order */
+    int TRANSACTION_OUT_OF_ORDER = 513;
+    /* Write exception */
+    int WRITE_EXCEPTION = 514;
+    /* Stream Unavailable */
+    int STREAM_UNAVAILABLE = 515;
+    /* Write cancelled exception */
+    int WRITE_CANCELLED_EXCEPTION = 516;
+    /* over-capacity/backpressure */
+    int OVER_CAPACITY = 517;
+
+    /** stream exists but is not ready (recovering etc.).
+     the difference between NOT_READY and UNAVAILABLE is that UNAVAILABLE
+     indicates the stream is no longer owned by the proxy and we should
+     redirect. NOT_READY indicates the stream exist at the proxy but isn't
+     eady for writes. */
+    int STREAM_NOT_READY = 518;
+    /* Region Unavailable */
+    int REGION_UNAVAILABLE = 519;
+    /* Invalid Enveloped Entry */
+    int INVALID_ENVELOPED_ENTRY = 520;
+    /* Unsupported metadata version */
+    int UNSUPPORTED_METADATA_VERSION = 521;
+    /* Log Already Exists */
+    int LOG_EXISTS = 522;
+    /* Checksum failed on the request */
+    int CHECKSUM_FAILED = 523;
+    /* Overcapacity: too many streams */
+    int TOO_MANY_STREAMS = 524;
+    /* Log Segment Not Found */
+    int LOG_SEGMENT_NOT_FOUND = 525;
+    /* End of Log Segment */
+    int END_OF_LOG_SEGMENT = 526;
+    /* Log Segment Is Truncated */
+    int LOG_SEGMENT_IS_TRUNCATED = 527;
+
+    /* 6xx: unexpected */
+
+    int UNEXPECTED = 600;
+    int INTERRUPTED = 601;
+    int INVALID_STREAM_NAME = 602;
+    int ILLEGAL_STATE = 603;
+
+    /* 10xx: reader exceptions */
+
+    int RETRYABLE_READ = 1000;
+    int LOG_READ_ERROR = 1001;
+    /* Read cancelled exception */
+    int READ_CANCELLED_EXCEPTION = 1002;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java
index ded9da1..a58a80f 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception is thrown when a log stream is not ready on server side for serving the write requests.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java
index fd0910e..ecba754 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Exception thrown when a stream is not available for serving traffic.
  */

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java
----------------------------------------------------------------------
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java
index 17e2e38..cc948d9 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.distributedlog.exceptions;
 
-import org.apache.distributedlog.thrift.service.StatusCode;
-
 /**
  * Signals that a server has been serving too many streams.
  */



[06/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
deleted file mode 100644
index 862f05a..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.util.Utils;
-import java.io.IOException;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.List;
-import java.util.TreeSet;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Transaction;
-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;
-
-/**
- * An implementation of the PlacementStateManager that saves data to and loads from Zookeeper to
- * avoid necessitating an additional system for the resource placement.
- */
-public class ZKPlacementStateManager implements PlacementStateManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(ZKPlacementStateManager.class);
-
-    private static final String SERVER_LOAD_DIR = "/.server-load";
-
-    private final String serverLoadPath;
-    private final ZooKeeperClient zkClient;
-
-    private boolean watching = false;
-
-    public ZKPlacementStateManager(URI uri, DistributedLogConfiguration conf, StatsLogger statsLogger) {
-        String zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
-        zkClient = BKNamespaceDriver.createZKClientBuilder(
-            String.format("ZKPlacementStateManager-%s", zkServers),
-            conf,
-            zkServers,
-            statsLogger.scope("placement_state_manager")).build();
-        serverLoadPath = uri.getPath() + SERVER_LOAD_DIR;
-    }
-
-    private void createServerLoadPathIfNoExists(byte[] data)
-        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
-        try {
-            Utils.zkCreateFullPathOptimistic(
-                zkClient, serverLoadPath, data, zkClient.getDefaultACL(), CreateMode.PERSISTENT);
-        } catch (KeeperException.NodeExistsException nee) {
-            logger.debug("the server load path {} is already created by others", serverLoadPath, nee);
-        }
-    }
-
-    @Override
-    public void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException {
-        logger.info("saving ownership");
-        try {
-            ZooKeeper zk = zkClient.get();
-            // use timestamp as data so watchers will see any changes
-            byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
-
-            if (zk.exists(serverLoadPath, false) == null) { //create path to rootnode if it does not yet exist
-                createServerLoadPathIfNoExists(timestamp);
-            }
-
-            Transaction tx = zk.transaction();
-            List<String> children = zk.getChildren(serverLoadPath, false);
-            HashSet<String> servers = new HashSet<String>(children);
-            tx.setData(serverLoadPath, timestamp, -1); // trigger the watcher that data has been updated
-            for (ServerLoad serverLoad : serverLoads) {
-                String server = serverToZkFormat(serverLoad.getServer());
-                String serverPath = serverPath(server);
-                if (servers.contains(server)) {
-                    servers.remove(server);
-                    tx.setData(serverPath, serverLoad.serialize(), -1);
-                } else {
-                    tx.create(serverPath, serverLoad.serialize(), zkClient.getDefaultACL(), CreateMode.PERSISTENT);
-                }
-            }
-            for (String server : servers) {
-                tx.delete(serverPath(server), -1);
-            }
-            tx.commit();
-        } catch (InterruptedException | IOException | KeeperException e) {
-            throw new StateManagerSaveException(e);
-        }
-    }
-
-    @Override
-    public TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException {
-        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
-        try {
-            ZooKeeper zk = zkClient.get();
-            List<String> children = zk.getChildren(serverLoadPath, false);
-            for (String server : children) {
-                ownerships.add(ServerLoad.deserialize(zk.getData(serverPath(server), false, new Stat())));
-            }
-            return ownerships;
-        } catch (InterruptedException | IOException | KeeperException e) {
-            throw new StateManagerLoadException(e);
-        }
-    }
-
-    @Override
-    public synchronized void watch(final PlacementCallback callback) {
-        if (watching) {
-            return; // do not double watch
-        }
-        watching = true;
-
-        try {
-            ZooKeeper zk = zkClient.get();
-            try {
-                zk.getData(serverLoadPath, new Watcher() {
-                    @Override
-                    public void process(WatchedEvent watchedEvent) {
-                        try {
-                            callback.callback(loadOwnership());
-                        } catch (StateManagerLoadException e) {
-                            logger.error("Watch of Ownership failed", e);
-                        } finally {
-                            watching = false;
-                            watch(callback);
-                        }
-                    }
-                }, new Stat());
-            } catch (KeeperException.NoNodeException nee) {
-                byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
-                createServerLoadPathIfNoExists(timestamp);
-                watching = false;
-                watch(callback);
-            }
-        } catch (ZooKeeperClient.ZooKeeperConnectionException | InterruptedException | KeeperException e) {
-            logger.error("Watch of Ownership failed", e);
-            watching = false;
-            watch(callback);
-        }
-    }
-
-    public String serverPath(String server) {
-        return String.format("%s/%s", serverLoadPath, server);
-    }
-
-    protected String serverToZkFormat(String server) {
-        return server.replaceAll("/", "--");
-    }
-
-    protected String zkFormatToServer(String zkFormattedServer) {
-        return zkFormattedServer.replaceAll("--", "/");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/package-info.java
deleted file mode 100644
index ea79251..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/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.
- */
-/**
- * Placement Policy to place streams across proxy services.
- */
-package org.apache.distributedlog.service.placement;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
deleted file mode 100644
index 83ac668..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.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 org.apache.distributedlog.service.stream;
-
-import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.exceptions.ChecksumFailedException;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Try;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-
-/**
- * Abstract Stream Operation.
- */
-public abstract class AbstractStreamOp<Response> implements StreamOp {
-
-    private static final Logger logger = LoggerFactory.getLogger(AbstractStreamOp.class);
-
-    protected final String stream;
-    protected final OpStatsLogger opStatsLogger;
-    private final Promise<Response> result = new Promise<Response>();
-    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
-    protected final Long checksum;
-    protected final Feature checksumDisabledFeature;
-
-    public AbstractStreamOp(String stream,
-                            OpStatsLogger statsLogger,
-                            Long checksum,
-                            Feature checksumDisabledFeature) {
-        this.stream = stream;
-        this.opStatsLogger = statsLogger;
-        // start here in case the operation is failed before executing.
-        stopwatch.reset().start();
-        this.checksum = checksum;
-        this.checksumDisabledFeature = checksumDisabledFeature;
-    }
-
-    @Override
-    public String streamName() {
-        return stream;
-    }
-
-    @Override
-    public Stopwatch stopwatch() {
-        return stopwatch;
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
-            Long serverChecksum = computeChecksum();
-            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
-                throw new ChecksumFailedException();
-            }
-        }
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return null;
-    }
-
-    @Override
-    public Future<Void> execute(AsyncLogWriter writer, Sequencer sequencer, Object txnLock) {
-        stopwatch.reset().start();
-        return executeOp(writer, sequencer, txnLock)
-                .addEventListener(new FutureEventListener<Response>() {
-            @Override
-            public void onSuccess(Response response) {
-                opStatsLogger.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                setResponse(response);
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-            }
-        }).voided();
-    }
-
-    /**
-     * Fail with current <i>owner</i> and its reason <i>t</i>.
-     *
-     * @param cause
-     *          failure reason
-     */
-    @Override
-    public void fail(Throwable cause) {
-        if (cause instanceof OwnershipAcquireFailedException) {
-            // Ownership exception is a control exception, not an error, so we don't stat
-            // it with the other errors.
-            OwnershipAcquireFailedException oafe = (OwnershipAcquireFailedException) cause;
-            fail(ResponseUtils.ownerToHeader(oafe.getCurrentOwner()));
-        } else {
-            opStatsLogger.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-            fail(ResponseUtils.exceptionToHeader(cause));
-        }
-    }
-
-    protected void setResponse(Response response) {
-      Return<Response> responseTry = new Return(response);
-      boolean isEmpty = result.updateIfEmpty(responseTry);
-      if (!isEmpty) {
-        Option<Try<Response>> resultTry = result.poll();
-        logger.error("Result set multiple times. Value='{}', New='{}'", resultTry, responseTry);
-      }
-    }
-
-    /**
-     * Return the full response, header and body.
-     *
-     * @return A future containing the response or the exception
-     *      encountered by the op if it failed.
-     */
-    public Future<Response> result() {
-        return result;
-    }
-
-    /**
-     * Execute the operation and return its corresponding response.
-     *
-     * @param writer
-     *          writer to execute the operation.
-     * @param sequencer
-     *          sequencer used for generating transaction id for stream operations
-     * @param txnLock
-     *          transaction lock to guarantee ordering of transaction id
-     * @return future representing the operation.
-     */
-    protected abstract Future<Response> executeOp(AsyncLogWriter writer,
-                                                  Sequencer sequencer,
-                                                  Object txnLock);
-
-    // fail the result with the given response header
-    protected abstract void fail(ResponseHeader header);
-
-    public static OpStatsLogger requestStat(StatsLogger statsLogger, String opName) {
-        return requestLogger(statsLogger).getOpStatsLogger(opName);
-    }
-
-    public static StatsLogger requestLogger(StatsLogger statsLogger) {
-        return statsLogger.scope("request");
-    }
-
-    public static StatsLogger requestScope(StatsLogger statsLogger, String scope) {
-        return requestLogger(statsLogger).scope(scope);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java
deleted file mode 100644
index 77c7d71..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.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 org.apache.distributedlog.service.stream;
-
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.ProtocolUtils;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Abstract Write Operation.
- */
-public abstract class AbstractWriteOp extends AbstractStreamOp<WriteResponse> {
-
-    protected AbstractWriteOp(String stream,
-                              OpStatsLogger statsLogger,
-                              Long checksum,
-                              Feature checksumDisabledFeature) {
-        super(stream, statsLogger, checksum, checksumDisabledFeature);
-    }
-
-    @Override
-    protected void fail(ResponseHeader header) {
-        setResponse(ResponseUtils.write(header));
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return ProtocolUtils.streamOpCRC32(stream);
-    }
-
-    @Override
-    public Future<ResponseHeader> responseHeader() {
-        return result().map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
-            @Override
-            public ResponseHeader apply(WriteResponse response) {
-                return response.getHeader();
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
deleted file mode 100644
index 6c98468..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.LogRecord;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.exceptions.AlreadyClosedException;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.LockingException;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.exceptions.RequestDeniedException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.distributedlog.thrift.service.BulkWriteResponse;
-import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.thrift.service.StatusCode;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.ConstFuture;
-import com.twitter.util.Future;
-import com.twitter.util.Future$;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Try;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Bulk Write Operation.
- */
-public class BulkWriteOp extends AbstractStreamOp<BulkWriteResponse> implements WriteOpWithPayload {
-    private final List<ByteBuffer> buffers;
-    private final long payloadSize;
-
-    // Stats
-    private final Counter deniedBulkWriteCounter;
-    private final Counter successRecordCounter;
-    private final Counter failureRecordCounter;
-    private final Counter redirectRecordCounter;
-    private final OpStatsLogger latencyStat;
-    private final Counter bytes;
-    private final Counter bulkWriteBytes;
-
-    private final AccessControlManager accessControlManager;
-
-    // We need to pass these through to preserve ownership change behavior in
-    // client/server. Only include failures which are guaranteed to have failed
-    // all subsequent writes.
-    private boolean isDefiniteFailure(Try<DLSN> result) {
-        boolean def = false;
-        try {
-            result.get();
-        } catch (Exception ex) {
-            if (ex instanceof OwnershipAcquireFailedException
-                || ex instanceof AlreadyClosedException
-                || ex instanceof LockingException) {
-                def = true;
-            }
-        }
-        return def;
-    }
-
-    public BulkWriteOp(String stream,
-                       List<ByteBuffer> buffers,
-                       StatsLogger statsLogger,
-                       StatsLogger perStreamStatsLogger,
-                       StreamPartitionConverter streamPartitionConverter,
-                       Long checksum,
-                       Feature checksumDisabledFeature,
-                       AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "bulkWrite"), checksum, checksumDisabledFeature);
-        this.buffers = buffers;
-        long total = 0;
-        // We do this here because the bytebuffers are mutable.
-        for (ByteBuffer bb : buffers) {
-          total += bb.remaining();
-        }
-        this.payloadSize = total;
-
-        final Partition partition = streamPartitionConverter.convert(stream);
-        // Write record stats
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedBulkWriteCounter = streamOpStats.requestDeniedCounter("bulkWrite");
-        this.successRecordCounter = streamOpStats.recordsCounter("success");
-        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
-        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
-        this.bulkWriteBytes = streamOpStats.scopedRequestCounter("bulkWrite", "bytes");
-        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "bulkWrite");
-        this.bytes = streamOpStats.streamRequestCounter(partition, "bulkWrite", "bytes");
-
-        this.accessControlManager = accessControlManager;
-
-        final long size = getPayloadSize();
-        result().addEventListener(new FutureEventListener<BulkWriteResponse>() {
-            @Override
-            public void onSuccess(BulkWriteResponse response) {
-                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
-                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                    bytes.add(size);
-                    bulkWriteBytes.add(size);
-                } else {
-                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                }
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowWrite(stream)) {
-            deniedBulkWriteCounter.inc();
-            throw new RequestDeniedException(stream, "bulkWrite");
-        }
-        super.preExecute();
-    }
-
-    @Override
-    public long getPayloadSize() {
-      return payloadSize;
-    }
-
-    @Override
-    protected Future<BulkWriteResponse> executeOp(AsyncLogWriter writer,
-                                                  Sequencer sequencer,
-                                                  Object txnLock) {
-        // Need to convert input buffers to LogRecords.
-        List<LogRecord> records;
-        Future<List<Future<DLSN>>> futureList;
-        synchronized (txnLock) {
-            records = asRecordList(buffers, sequencer);
-            futureList = writer.writeBulk(records);
-        }
-
-        // Collect into a list of tries to make it easier to extract exception or DLSN.
-        Future<List<Try<DLSN>>> writes = asTryList(futureList);
-
-        Future<BulkWriteResponse> response = writes.flatMap(
-            new AbstractFunction1<List<Try<DLSN>>, Future<BulkWriteResponse>>() {
-                @Override
-                public Future<BulkWriteResponse> apply(List<Try<DLSN>> results) {
-
-                    // Considered a success at batch level even if no individual writes succeeed.
-                    // The reason is that its impossible to make an appropriate decision re retries without
-                    // individual buffer failure reasons.
-                    List<WriteResponse> writeResponses = new ArrayList<WriteResponse>(results.size());
-                    BulkWriteResponse bulkWriteResponse =
-                        ResponseUtils.bulkWriteSuccess().setWriteResponses(writeResponses);
-
-                    // Promote the first result to an op-level failure if we're sure all other writes have
-                    // failed.
-                    if (results.size() > 0) {
-                        Try<DLSN> firstResult = results.get(0);
-                        if (isDefiniteFailure(firstResult)) {
-                            return new ConstFuture(firstResult);
-                        }
-                    }
-
-                    // Translate all futures to write responses.
-                    Iterator<Try<DLSN>> iterator = results.iterator();
-                    while (iterator.hasNext()) {
-                        Try<DLSN> completedFuture = iterator.next();
-                        try {
-                            DLSN dlsn = completedFuture.get();
-                            WriteResponse writeResponse = ResponseUtils.writeSuccess().setDlsn(dlsn.serialize());
-                            writeResponses.add(writeResponse);
-                            successRecordCounter.inc();
-                        } catch (Exception ioe) {
-                            WriteResponse writeResponse = ResponseUtils.write(ResponseUtils.exceptionToHeader(ioe));
-                            writeResponses.add(writeResponse);
-                            if (StatusCode.FOUND == writeResponse.getHeader().getCode()) {
-                                redirectRecordCounter.inc();
-                            } else {
-                                failureRecordCounter.inc();
-                            }
-                        }
-                    }
-
-                    return Future.value(bulkWriteResponse);
-                }
-            }
-        );
-
-        return response;
-    }
-
-    private List<LogRecord> asRecordList(List<ByteBuffer> buffers, Sequencer sequencer) {
-        List<LogRecord> records = new ArrayList<LogRecord>(buffers.size());
-        for (ByteBuffer buffer : buffers) {
-            byte[] payload = new byte[buffer.remaining()];
-            buffer.get(payload);
-            records.add(new LogRecord(sequencer.nextId(), payload));
-        }
-        return records;
-    }
-
-    private Future<List<Try<DLSN>>> asTryList(Future<List<Future<DLSN>>> futureList) {
-        return futureList.flatMap(new AbstractFunction1<List<Future<DLSN>>, Future<List<Try<DLSN>>>>() {
-            @Override
-            public Future<List<Try<DLSN>>> apply(List<Future<DLSN>> results) {
-                return Future$.MODULE$.collectToTry(results);
-            }
-        });
-    }
-
-    @Override
-    protected void fail(ResponseHeader header) {
-        if (StatusCode.FOUND == header.getCode()) {
-            redirectRecordCounter.add(buffers.size());
-        } else {
-            failureRecordCounter.add(buffers.size());
-        }
-        setResponse(ResponseUtils.bulkWrite(header));
-    }
-
-    @Override
-    public Future<ResponseHeader> responseHeader() {
-        return result().map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
-            @Override
-            public ResponseHeader apply(BulkWriteResponse response) {
-                return response.getHeader();
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
deleted file mode 100644
index 3ecb46f..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.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 org.apache.distributedlog.service.stream;
-
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.RequestDeniedException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Operation to delete a log stream.
- */
-public class DeleteOp extends AbstractWriteOp {
-    private final StreamManager streamManager;
-    private final Counter deniedDeleteCounter;
-    private final AccessControlManager accessControlManager;
-
-    public DeleteOp(String stream,
-                    StatsLogger statsLogger,
-                    StatsLogger perStreamStatsLogger,
-                    StreamManager streamManager,
-                    Long checksum,
-                    Feature checksumEnabledFeature,
-                    AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "delete"), checksum, checksumEnabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedDeleteCounter = streamOpStats.requestDeniedCounter("delete");
-        this.accessControlManager = accessControlManager;
-        this.streamManager = streamManager;
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        Future<Void> result = streamManager.deleteAndRemoveAsync(streamName());
-        return result.map(new AbstractFunction1<Void, WriteResponse>() {
-            @Override
-            public WriteResponse apply(Void value) {
-                return ResponseUtils.writeSuccess();
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowTruncate(stream)) {
-            deniedDeleteCounter.inc();
-            throw new RequestDeniedException(stream, "delete");
-        }
-        super.preExecute();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
deleted file mode 100644
index 0ffa619..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.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 org.apache.distributedlog.service.stream;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.LogRecord;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.RequestDeniedException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Heartbeat Operation.
- */
-public class HeartbeatOp extends AbstractWriteOp {
-
-    static final byte[] HEARTBEAT_DATA = "heartbeat".getBytes(UTF_8);
-
-    private final AccessControlManager accessControlManager;
-    private final Counter deniedHeartbeatCounter;
-    private final byte dlsnVersion;
-
-    private boolean writeControlRecord = false;
-
-    public HeartbeatOp(String stream,
-                       StatsLogger statsLogger,
-                       StatsLogger perStreamStatsLogger,
-                       byte dlsnVersion,
-                       Long checksum,
-                       Feature checksumDisabledFeature,
-                       AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "heartbeat"), checksum, checksumDisabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedHeartbeatCounter = streamOpStats.requestDeniedCounter("heartbeat");
-        this.dlsnVersion = dlsnVersion;
-        this.accessControlManager = accessControlManager;
-    }
-
-    public HeartbeatOp setWriteControlRecord(boolean writeControlRecord) {
-        this.writeControlRecord = writeControlRecord;
-        return this;
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        // write a control record if heartbeat is the first request of the recovered log segment.
-        if (writeControlRecord) {
-            long txnId;
-            Future<DLSN> writeResult;
-            synchronized (txnLock) {
-                txnId = sequencer.nextId();
-                LogRecord hbRecord = new LogRecord(txnId, HEARTBEAT_DATA);
-                hbRecord.setControl();
-                writeResult = writer.write(hbRecord);
-            }
-            return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
-                @Override
-                public WriteResponse apply(DLSN value) {
-                    return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
-                }
-            });
-        } else {
-            return Future.value(ResponseUtils.writeSuccess());
-        }
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowAcquire(stream)) {
-            deniedHeartbeatCounter.inc();
-            throw new RequestDeniedException(stream, "heartbeat");
-        }
-        super.preExecute();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
deleted file mode 100644
index 6ec8642..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.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 org.apache.distributedlog.service.stream;
-
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.exceptions.DLException;
-import org.apache.distributedlog.exceptions.RequestDeniedException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Operation to release ownership of a log stream.
- */
-public class ReleaseOp extends AbstractWriteOp {
-    private final StreamManager streamManager;
-    private final Counter deniedReleaseCounter;
-    private final AccessControlManager accessControlManager;
-
-    public ReleaseOp(String stream,
-                     StatsLogger statsLogger,
-                     StatsLogger perStreamStatsLogger,
-                     StreamManager streamManager,
-                     Long checksum,
-                     Feature checksumDisabledFeature,
-                     AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "release"), checksum, checksumDisabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedReleaseCounter = streamOpStats.requestDeniedCounter("release");
-        this.accessControlManager = accessControlManager;
-        this.streamManager = streamManager;
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        Future<Void> result = streamManager.closeAndRemoveAsync(streamName());
-        return result.map(new AbstractFunction1<Void, WriteResponse>() {
-            @Override
-            public WriteResponse apply(Void value) {
-                return ResponseUtils.writeSuccess();
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowRelease(stream)) {
-            deniedReleaseCounter.inc();
-            throw new RequestDeniedException(stream, "release");
-        }
-        super.preExecute();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/Stream.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/Stream.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/Stream.java
deleted file mode 100644
index 3517a63..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/Stream.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 org.apache.distributedlog.service.stream;
-
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.service.streamset.Partition;
-import com.twitter.util.Future;
-import java.io.IOException;
-
-/**
- * Stream is the per stream request handler in the DL service layer.
- *
- * <p>The collection of Streams in the proxy are managed by StreamManager.
- */
-public interface Stream {
-
-    /**
-     * Get the stream configuration for this stream.
-     *
-     * @return stream configuration
-     */
-    DynamicDistributedLogConfiguration getStreamConfiguration();
-
-    /**
-     * Get the stream's last recorded current owner (may be out of date). Used
-     * as a hint for the client.
-     * @return last known owner for the stream
-     */
-    String getOwner();
-
-    /**
-     * Get the stream name.
-     * @return stream name
-     */
-    String getStreamName();
-
-    /**
-     * Get the represented partition name.
-     *
-     * @return represented partition name.
-     */
-    Partition getPartition();
-
-    /**
-     * Expensive initialization code run after stream has been allocated in
-     * StreamManager.
-     *
-     * @throws IOException when encountered exception on initialization
-     */
-    void initialize() throws IOException;
-
-    /**
-     * Another initialize method (actually Thread.start). Should probably be
-     * moved to initialize().
-     */
-    void start();
-
-    /**
-     * Asynchronous close method.
-     * @param reason for closing
-     * @return future satisfied once close complete
-     */
-    Future<Void> requestClose(String reason);
-
-    /**
-     * Delete the stream from DL backend.
-     *
-     * @throws IOException when encountered exception on deleting the stream.
-     */
-    void delete() throws IOException;
-
-    /**
-     * Execute the stream operation against this stream.
-     *
-     * @param op operation to execute
-     */
-    void submit(StreamOp op);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java
deleted file mode 100644
index 845ef21..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.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 org.apache.distributedlog.service.stream;
-
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-
-/**
- * Factory to create a stream with provided stream configuration {@code streamConf}.
- */
-public interface StreamFactory {
-
-    /**
-     * Create a stream object.
-     *
-     * @param name stream name
-     * @param streamConf stream configuration
-     * @param streamManager manager of streams
-     * @return stream object
-     */
-    Stream create(String name,
-                  DynamicDistributedLogConfiguration streamConf,
-                  StreamManager streamManager);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
deleted file mode 100644
index 2b90d55..0000000
--- a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.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 org.apache.distributedlog.service.stream;
-
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.service.FatalErrorHandler;
-import org.apache.distributedlog.service.config.ServerConfiguration;
-import org.apache.distributedlog.service.config.StreamConfigProvider;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Timer;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.jboss.netty.util.HashedWheelTimer;
-
-/**
- * The implementation of {@link StreamFactory}.
- */
-public class StreamFactoryImpl implements StreamFactory {
-    private final String clientId;
-    private final StreamOpStats streamOpStats;
-    private final ServerConfiguration serverConfig;
-    private final DistributedLogConfiguration dlConfig;
-    private final FeatureProvider featureProvider;
-    private final StreamConfigProvider streamConfigProvider;
-    private final StreamPartitionConverter streamPartitionConverter;
-    private final DistributedLogNamespace dlNamespace;
-    private final OrderedScheduler scheduler;
-    private final FatalErrorHandler fatalErrorHandler;
-    private final HashedWheelTimer requestTimer;
-    private final Timer futureTimer;
-
-    public StreamFactoryImpl(String clientId,
-        StreamOpStats streamOpStats,
-        ServerConfiguration serverConfig,
-        DistributedLogConfiguration dlConfig,
-        FeatureProvider featureProvider,
-        StreamConfigProvider streamConfigProvider,
-        StreamPartitionConverter streamPartitionConverter,
-        DistributedLogNamespace dlNamespace,
-        OrderedScheduler scheduler,
-        FatalErrorHandler fatalErrorHandler,
-        HashedWheelTimer requestTimer) {
-
-        this.clientId = clientId;
-        this.streamOpStats = streamOpStats;
-        this.serverConfig = serverConfig;
-        this.dlConfig = dlConfig;
-        this.featureProvider = featureProvider;
-        this.streamConfigProvider = streamConfigProvider;
-        this.streamPartitionConverter = streamPartitionConverter;
-        this.dlNamespace = dlNamespace;
-        this.scheduler = scheduler;
-        this.fatalErrorHandler = fatalErrorHandler;
-        this.requestTimer = requestTimer;
-        this.futureTimer = new com.twitter.finagle.util.HashedWheelTimer(requestTimer);
-    }
-
-    @Override
-    public Stream create(String name,
-                         DynamicDistributedLogConfiguration streamConf,
-                         StreamManager streamManager) {
-        return new StreamImpl(name,
-            streamPartitionConverter.convert(name),
-            clientId,
-            streamManager,
-            streamOpStats,
-            serverConfig,
-            dlConfig,
-            streamConf,
-            featureProvider,
-            streamConfigProvider,
-            dlNamespace,
-            scheduler,
-            fatalErrorHandler,
-            requestTimer,
-            futureTimer);
-    }
-}


[02/30] incubator-distributedlog git commit: DL-205: Remove StatusCode dependency on DLException

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
deleted file mode 100644
index d0a2f88..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.DistributedLogConfiguration;
-import org.apache.distributedlog.feature.DefaultFeatureProvider;
-import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-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.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test Case for {@link org.apache.distributedlog.exceptions.RegionUnavailableException}.
- */
-public class TestRegionUnavailable extends DistributedLogServerTestCase {
-
-    /**
-     * A feature provider for testing.
-     */
-    public static class TestFeatureProvider extends DefaultFeatureProvider {
-
-        public TestFeatureProvider(String rootScope,
-                                   DistributedLogConfiguration conf,
-                                   StatsLogger statsLogger) {
-            super(rootScope, conf, statsLogger);
-        }
-
-        @Override
-        protected Feature makeFeature(String featureName) {
-            if (featureName.contains(ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase())) {
-                return new SettableFeature(featureName, 10000);
-            }
-            return super.makeFeature(featureName);
-        }
-
-        @Override
-        protected FeatureProvider makeProvider(String fullScopeName) {
-            return super.makeProvider(fullScopeName);
-        }
-    }
-
-    private final int numServersPerDC = 3;
-    private final List<DLServer> localCluster;
-    private final List<DLServer> remoteCluster;
-    private TwoRegionDLClient client;
-
-    public TestRegionUnavailable() {
-        super(true);
-        this.localCluster = new ArrayList<DLServer>();
-        this.remoteCluster = new ArrayList<DLServer>();
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        DistributedLogConfiguration localConf = new DistributedLogConfiguration();
-        localConf.addConfiguration(conf);
-        localConf.setFeatureProviderClass(TestFeatureProvider.class);
-        DistributedLogConfiguration remoteConf = new DistributedLogConfiguration();
-        remoteConf.addConfiguration(conf);
-        super.setup();
-        int localPort = 9010;
-        int remotePort = 9020;
-        for (int i = 0; i < numServersPerDC; i++) {
-            localCluster.add(createDistributedLogServer(localConf, localPort + i));
-            remoteCluster.add(createDistributedLogServer(remoteConf, remotePort + i));
-        }
-        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
-        for (DLServer server : localCluster) {
-            regionMap.put(server.getAddress(), "local");
-        }
-        for (DLServer server : remoteCluster) {
-            regionMap.put(server.getAddress(), "remote");
-        }
-        client = createTwoRegionDLClient("two_regions_client", regionMap);
-
-    }
-
-    private void registerStream(String streamName) {
-        for (DLServer server : localCluster) {
-            client.localRoutingService.addHost(streamName, server.getAddress());
-        }
-        client.remoteRoutingService.addHost(streamName, remoteCluster.get(0).getAddress());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != client) {
-            client.shutdown();
-        }
-        for (DLServer server : localCluster) {
-            server.shutdown();
-        }
-        for (DLServer server : remoteCluster) {
-            server.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testRegionUnavailable() throws Exception {
-        String name = "dlserver-region-unavailable";
-        registerStream(name);
-
-        for (long i = 1; i <= 10; i++) {
-            client.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
-        }
-
-        // check local region
-        for (DLServer server : localCluster) {
-            checkStreams(0, server);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java
deleted file mode 100644
index c8b8bdf..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestStatsFilter.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 org.apache.distributedlog.service;
-
-import static org.junit.Assert.assertEquals;
-
-import com.twitter.finagle.Service;
-import com.twitter.finagle.service.ConstantService;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StatsFilter}.
- */
-public class TestStatsFilter {
-
-    class RuntimeExService<Req, Rep> extends Service<Req, Rep> {
-        public Future<Rep> apply(Req request) {
-            throw new RuntimeException("test");
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceSuccess() throws Exception {
-        StatsLogger stats = new NullStatsLogger();
-        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
-        Future<String> result = filter.apply("", new ConstantService<String, String>(Future.value("result")));
-        assertEquals("result", Await.result(result));
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceFailure() throws Exception {
-        StatsLogger stats = new NullStatsLogger();
-        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
-        try {
-            filter.apply("", new RuntimeExService<String, String>());
-        } catch (RuntimeException ex) {
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java
deleted file mode 100644
index 21bebb5..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.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 org.apache.distributedlog.service.balancer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.junit.Test;
-
-/**
- * Test Case for {@link BalancerUtils}.
- */
-public class TestBalancerUtils {
-
-    @Test(timeout = 60000)
-    public void testCalculateNumStreamsToRebalance() {
-        String myNode = "mynode";
-
-        // empty load distribution
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, new HashMap<String, Integer>(), 0, 10));
-        // my node doesn't exist in load distribution
-        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
-        loadDistribution.put("node1", 10);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 0, 10));
-        // my node doesn't reach rebalance water mark
-        loadDistribution.clear();
-        loadDistribution.put("node1", 1);
-        loadDistribution.put(myNode, 100);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 200, 10));
-        // my node is below average in the cluster.
-        loadDistribution.clear();
-        loadDistribution.put(myNode, 1);
-        loadDistribution.put("node1", 99);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 0, 10));
-        // my node is above average in the cluster
-        assertEquals(49, BalancerUtils.calculateNumStreamsToRebalance(
-                "node1", loadDistribution, 0, 10));
-        // my node is at the tolerance range
-        loadDistribution.clear();
-        loadDistribution.put(myNode, 55);
-        loadDistribution.put("node1", 45);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 0, 10));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
deleted file mode 100644
index fb3fb6e..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.fail;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.distributedlog.service.DLSocketAddress;
-import org.apache.distributedlog.service.DistributedLogClient;
-import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
-import org.apache.distributedlog.service.DistributedLogServerTestCase;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.commons.lang3.tuple.Pair;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link ClusterBalancer}.
- */
-public class TestClusterBalancer extends DistributedLogServerTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestClusterBalancer.class);
-
-    private final int numServers = 5;
-    private final List<DLServer> cluster;
-    private DLClient client;
-
-    public TestClusterBalancer() {
-        super(true);
-        this.cluster = new ArrayList<DLServer>();
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        int initPort = 9001;
-        for (int i = 0; i < numServers; i++) {
-            cluster.add(createDistributedLogServer(initPort + i));
-        }
-        client = createDistributedLogClient("cluster_client", Optional.<String>absent());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != client) {
-            client.shutdown();
-        }
-        for (DLServer server: cluster) {
-            server.shutdown();
-        }
-    }
-
-    private void initStreams(String namePrefix) {
-        logger.info("Init streams with prefix {}", namePrefix);
-        // Stream Distribution: 5, 4, 3, 2, 1
-        initStreams(namePrefix, 5, 1, 0);
-        initStreams(namePrefix, 4, 6, 1);
-        initStreams(namePrefix, 3, 10, 2);
-        initStreams(namePrefix, 2, 13, 3);
-        initStreams(namePrefix, 1, 15, 4);
-    }
-
-    private void initStreams(String namePrefix, int numStreams, int streamId, int proxyId) {
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + (streamId++);
-            client.routingService.addHost(name, cluster.get(proxyId).getAddress());
-        }
-    }
-
-    private void writeStreams(String namePrefix) throws Exception {
-        logger.info("Write streams with prefix {}", namePrefix);
-        writeStreams(namePrefix, 5, 1);
-        writeStreams(namePrefix, 4, 6);
-        writeStreams(namePrefix, 3, 10);
-        writeStreams(namePrefix, 2, 13);
-        writeStreams(namePrefix, 1, 15);
-    }
-
-    private void writeStreams(String namePrefix, int numStreams, int streamId) throws Exception {
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + (streamId++);
-            try {
-                Await.result(client.dlClient.write(name, ByteBuffer.wrap(name.getBytes(UTF_8))));
-            } catch (Exception e) {
-                logger.error("Error writing stream {} : ", name, e);
-                throw e;
-            }
-        }
-    }
-
-    private void validateStreams(String namePrefix) throws Exception {
-        logger.info("Validate streams with prefix {}", namePrefix);
-        validateStreams(namePrefix, 5, 1, 0);
-        validateStreams(namePrefix, 4, 6, 1);
-        validateStreams(namePrefix, 3, 10, 2);
-        validateStreams(namePrefix, 2, 13, 3);
-        validateStreams(namePrefix, 1, 15, 4);
-    }
-
-    private void validateStreams(String namePrefix, int numStreams, int streamId, int proxyIdx) {
-        Set<String> expectedStreams = new HashSet<String>();
-        for (int i = 0; i < numStreams; i++) {
-            expectedStreams.add(namePrefix + (streamId++));
-        }
-        checkStreams(expectedStreams, cluster.get(proxyIdx));
-    }
-
-    @Ignore
-    @Test(timeout = 60000)
-    public void testBalanceAll() throws Exception {
-        String namePrefix = "clusterbalancer-balance-all-";
-
-        initStreams(namePrefix);
-        writeStreams(namePrefix);
-        validateStreams(namePrefix);
-
-        Optional<RateLimiter> rateLimiter = Optional.absent();
-
-        Balancer balancer = new ClusterBalancer(client.dlClientBuilder,
-                Pair.of((DistributedLogClient) client.dlClient, (MonitorServiceClient) client.dlClient));
-        logger.info("Rebalancing from 'unknown' target");
-        try {
-            balancer.balanceAll("unknown", 10, rateLimiter);
-            fail("Should fail on balanceAll from 'unknown' target.");
-        } catch (IllegalArgumentException iae) {
-            // expected
-        }
-        validateStreams(namePrefix);
-
-        logger.info("Rebalancing from 'unexisted' host");
-        String addr = DLSocketAddress.toString(DLSocketAddress.getSocketAddress(9999));
-        balancer.balanceAll(addr, 10, rateLimiter);
-        validateStreams(namePrefix);
-
-        addr = DLSocketAddress.toString(cluster.get(0).getAddress());
-        logger.info("Rebalancing from host {}.", addr);
-        balancer.balanceAll(addr, 10, rateLimiter);
-        checkStreams(0, cluster.get(0));
-        checkStreams(4, cluster.get(1));
-        checkStreams(3, cluster.get(2));
-        checkStreams(4, cluster.get(3));
-        checkStreams(4, cluster.get(4));
-
-        addr = DLSocketAddress.toString(cluster.get(2).getAddress());
-        logger.info("Rebalancing from host {}.", addr);
-        balancer.balanceAll(addr, 10, rateLimiter);
-        checkStreams(3, cluster.get(0));
-        checkStreams(4, cluster.get(1));
-        checkStreams(0, cluster.get(2));
-        checkStreams(4, cluster.get(3));
-        checkStreams(4, cluster.get(4));
-
-        logger.info("Rebalancing the cluster");
-        balancer.balance(0, 0.0f, 10, rateLimiter);
-        for (int i = 0; i < 5; i++) {
-            checkStreams(3, cluster.get(i));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
deleted file mode 100644
index 6734083..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Sets;
-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 org.junit.Test;
-
-/**
- * Test Case for {@link CountBasedStreamChooser}.
- */
-public class TestCountBasedStreamChooser {
-
-    @Test(timeout = 60000)
-    public void testEmptyStreamDistribution() {
-        try {
-            new CountBasedStreamChooser(new HashMap<SocketAddress, Set<String>>());
-            fail("Should fail constructing stream chooser if the stream distribution is empty");
-        } catch (IllegalArgumentException iae) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testMultipleHostsWithEmptyStreams() {
-        for (int i = 1; i <= 3; i++) {
-            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-            int port = 1000;
-            for (int j = 0; j < i; j++) {
-                SocketAddress address = new InetSocketAddress("127.0.0.1", port + j);
-                streamDistribution.put(address, new HashSet<String>());
-            }
-
-            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-            for (int k = 0; k < i + 1; k++) {
-                assertNull(chooser.choose());
-            }
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testSingleHostWithStreams() {
-        for (int i = 0; i < 3; i++) {
-            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-
-            Set<String> streams = new HashSet<String>();
-            for (int j = 0; j < 3; j++) {
-                streams.add("SingleHostStream-" + j);
-            }
-
-            int port = 1000;
-            SocketAddress address = new InetSocketAddress("127.0.0.1", port);
-            streamDistribution.put(address, streams);
-
-            for (int k = 1; k <= i; k++) {
-                address = new InetSocketAddress("127.0.0.1", port + k);
-                streamDistribution.put(address, new HashSet<String>());
-            }
-
-            Set<String> choosenStreams = new HashSet<String>();
-
-            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-            for (int l = 0; l < 3 + i + 1; l++) {
-                String s = chooser.choose();
-                if (null != s) {
-                    choosenStreams.add(s);
-                }
-            }
-
-            assertEquals(streams.size(), choosenStreams.size());
-            assertTrue(Sets.difference(streams, choosenStreams).immutableCopy().isEmpty());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testHostsHaveSameNumberStreams() {
-        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-        Set<String> allStreams = new HashSet<String>();
-
-        int numHosts = 3;
-        int numStreamsPerHost = 3;
-
-        int port = 1000;
-        for (int i = 1; i <= numHosts; i++) {
-            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
-            Set<String> streams = new HashSet<String>();
-
-            for (int j = 1; j <= numStreamsPerHost; j++) {
-                String streamName = "HostsHaveSameNumberStreams-" + i + "-" + j;
-                streams.add(streamName);
-                allStreams.add(streamName);
-            }
-
-            streamDistribution.put(address, streams);
-        }
-
-        Set<String> streamsChoosen = new HashSet<String>();
-        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-        for (int i = 1; i <= numStreamsPerHost; i++) {
-            for (int j = 1; j <= numHosts; j++) {
-                String s = chooser.choose();
-                assertNotNull(s);
-                streamsChoosen.add(s);
-            }
-            for (int j = 0; j < numHosts; j++) {
-                assertEquals(numStreamsPerHost - i, chooser.streamsDistribution.get(j).getRight().size());
-            }
-        }
-        assertNull(chooser.choose());
-        assertEquals(numHosts * numStreamsPerHost, streamsChoosen.size());
-        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testHostsHaveDifferentNumberStreams() {
-        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-        Set<String> allStreams = new HashSet<String>();
-
-        int numHosts = 6;
-        int maxStreamsPerHost = 4;
-
-        int port = 1000;
-        for (int i = 0; i < numHosts; i++) {
-            int group = i / 2;
-            int numStreamsThisGroup = maxStreamsPerHost - group;
-
-            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
-            Set<String> streams = new HashSet<String>();
-
-            for (int j = 1; j <= numStreamsThisGroup; j++) {
-                String streamName = "HostsHaveDifferentNumberStreams-" + i + "-" + j;
-                streams.add(streamName);
-                allStreams.add(streamName);
-            }
-
-            streamDistribution.put(address, streams);
-        }
-
-        Set<String> streamsChoosen = new HashSet<String>();
-        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-
-        for (int i = 0; i < allStreams.size(); i++) {
-            String s = chooser.choose();
-            assertNotNull(s);
-            streamsChoosen.add(s);
-        }
-        assertNull(chooser.choose());
-        assertEquals(allStreams.size(), streamsChoosen.size());
-        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testLimitedStreamChooser() {
-        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-
-        Set<String> streams = new HashSet<String>();
-        for (int j = 0; j < 10; j++) {
-            streams.add("SingleHostStream-" + j);
-        }
-
-        int port = 1000;
-        SocketAddress address = new InetSocketAddress("127.0.0.1", port);
-        streamDistribution.put(address, streams);
-
-        Set<String> choosenStreams = new HashSet<String>();
-
-        CountBasedStreamChooser underlying = new CountBasedStreamChooser(streamDistribution);
-        LimitedStreamChooser chooser = LimitedStreamChooser.of(underlying, 1);
-        for (int l = 0; l < 10; l++) {
-            String s = chooser.choose();
-            if (null != s) {
-                choosenStreams.add(s);
-            }
-        }
-
-        assertEquals(1, choosenStreams.size());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java
deleted file mode 100644
index 73fa98a..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.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 org.apache.distributedlog.service.balancer;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
-import org.apache.distributedlog.service.DistributedLogServerTestCase;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import java.util.Set;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link SimpleBalancer}.
- */
-public class TestSimpleBalancer extends DistributedLogServerTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestSimpleBalancer.class);
-
-    DLClient targetClient;
-    DLServer targetServer;
-
-    public TestSimpleBalancer() {
-        super(true);
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        targetServer = createDistributedLogServer(7003);
-        targetClient = createDistributedLogClient("target", Optional.<String>absent());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != targetClient) {
-            targetClient.shutdown();
-        }
-        if (null != targetServer) {
-            targetServer.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBalanceAll() throws Exception {
-        String namePrefix = "simplebalancer-balance-all-";
-        int numStreams = 10;
-
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            // src client
-            dlClient.routingService.addHost(name, dlServer.getAddress());
-            // target client
-            targetClient.routingService.addHost(name, targetServer.getAddress());
-        }
-
-        // write to multiple streams
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
-        }
-
-        // validation
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-        }
-
-        Optional<RateLimiter> rateLimiter = Optional.absent();
-
-        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
-                                               "target", targetClient.dlClient, targetClient.dlClient);
-        logger.info("Rebalancing from 'unknown' target");
-        try {
-            balancer.balanceAll("unknown", 10, rateLimiter);
-            fail("Should fail on balanceAll from 'unknown' target.");
-        } catch (IllegalArgumentException iae) {
-            // expected
-        }
-
-        // nothing to balance from 'target'
-        logger.info("Rebalancing from 'target' target");
-        balancer.balanceAll("target", 1, rateLimiter);
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-        }
-
-        // balance all streams from 'source'
-        logger.info("Rebalancing from 'source' target");
-        balancer.balanceAll("source", 10, rateLimiter);
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, targetClient, targetServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBalanceStreams() throws Exception {
-        String namePrefix = "simplebalancer-balance-streams-";
-        int numStreams = 10;
-
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            // src client
-            dlClient.routingService.addHost(name, dlServer.getAddress());
-            // target client
-            targetClient.routingService.addHost(name, targetServer.getAddress());
-        }
-
-        // write to multiple streams
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
-        }
-
-        // validation
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-        }
-
-        Optional<RateLimiter> rateLimiter = Optional.absent();
-
-        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
-                                               "target", targetClient.dlClient, targetClient.dlClient);
-
-        // balance all streams from 'source'
-        logger.info("Rebalancing streams between targets");
-        balancer.balance(0, 0, 10, rateLimiter);
-
-        Set<String> sourceStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(dlClient));
-        Set<String> targetStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(targetClient));
-
-        assertEquals(numStreams / 2, sourceStreams.size());
-        assertEquals(numStreams / 2, targetStreams.size());
-
-        for (String name : sourceStreams) {
-            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, true, true);
-            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, false, false);
-        }
-
-        for (String name : targetStreams) {
-            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, true, true);
-            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, false, false);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
deleted file mode 100644
index ce7b2c1..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.balancer;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.base.Optional;
-import org.apache.distributedlog.service.DistributedLogClient;
-import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
-import org.apache.distributedlog.service.DistributedLogServerTestCase;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StreamMover}.
- */
-public class TestStreamMover extends DistributedLogServerTestCase {
-
-    DLClient targetClient;
-    DLServer targetServer;
-
-    public TestStreamMover() {
-        super(true);
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        targetServer = createDistributedLogServer(7003);
-        targetClient = createDistributedLogClient("target", Optional.<String>absent());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != targetClient) {
-            targetClient.shutdown();
-        }
-        if (null != targetServer) {
-            targetServer.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testMoveStream() throws Exception {
-        String name = "dlserver-move-stream";
-
-        // src client
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-        // target client
-        targetClient.routingService.addHost(name, targetServer.getAddress());
-
-        // src client write a record to that stream
-        Await.result(((DistributedLogClient) dlClient.dlClient).write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-        checkStream(name, dlClient, dlServer, 1, 1, 1, true, true);
-        checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-
-        StreamMover streamMover = new StreamMoverImpl("source", dlClient.dlClient, dlClient.dlClient,
-                                                      "target", targetClient.dlClient, targetClient.dlClient);
-        assertTrue(streamMover.moveStream(name));
-        checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
-        checkStream(name, targetClient, targetServer, 1, 1, 1, true, true);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java
deleted file mode 100644
index 71dfa45..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.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 org.apache.distributedlog.service.config;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServerConfiguration}.
- */
-public class TestServerConfiguration {
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testUnassignedShardId() {
-        new ServerConfiguration().validate();
-    }
-
-    @Test(timeout = 60000)
-    public void testAssignedShardId() {
-        ServerConfiguration conf = new ServerConfiguration();
-        conf.setServerShardId(100);
-        conf.validate();
-        assertEquals(100, conf.getServerShardId());
-    }
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testInvalidServerThreads() {
-        ServerConfiguration conf = new ServerConfiguration();
-        conf.setServerShardId(100);
-        conf.setServerThreads(-1);
-        conf.validate();
-    }
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testInvalidDlsnVersion() {
-        ServerConfiguration conf = new ServerConfiguration();
-        conf.setServerShardId(100);
-        conf.setDlsnVersion((byte) 9999);
-        conf.validate();
-    }
-
-    @Test(timeout = 60000)
-    public void testUseHostnameAsAllocatorPoolName() {
-        ServerConfiguration conf = new ServerConfiguration();
-        assertFalse("Should not use hostname by default", conf.isUseHostnameAsAllocatorPoolName());
-        conf.setUseHostnameAsAllocatorPoolName(true);
-        assertTrue("Should use hostname now", conf.isUseHostnameAsAllocatorPoolName());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
deleted file mode 100644
index bdbde11..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.config;
-
-import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.config.PropertiesWriter;
-import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StreamConfigProvider}.
- */
-public class TestStreamConfigProvider {
-    private static final String DEFAULT_CONFIG_DIR = "conf";
-    private final String defaultConfigPath;
-    private final ScheduledExecutorService configExecutorService;
-
-    public TestStreamConfigProvider() throws Exception {
-        this.configExecutorService = Executors.newScheduledThreadPool(1,
-                new ThreadFactoryBuilder().setNameFormat("DistributedLogService-Dyncfg-%d").build());
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.save();
-        this.defaultConfigPath = writer.getFile().getPath();
-    }
-
-    StreamConfigProvider getServiceProvider(StreamPartitionConverter converter)
-            throws Exception {
-        return getServiceProvider(converter, DEFAULT_CONFIG_DIR);
-    }
-
-    StreamConfigProvider getServiceProvider(
-            StreamPartitionConverter converter,
-            String configPath,
-            String defaultPath) throws Exception {
-        return new ServiceStreamConfigProvider(
-                configPath,
-                defaultPath,
-                converter,
-                configExecutorService,
-                1,
-                TimeUnit.SECONDS);
-    }
-
-    StreamConfigProvider getServiceProvider(
-            StreamPartitionConverter converter,
-            String configPath) throws Exception {
-        return getServiceProvider(converter, configPath, defaultConfigPath);
-    }
-
-    StreamConfigProvider getDefaultProvider(String configFile) throws Exception {
-        return new DefaultStreamConfigProvider(configFile, configExecutorService, 1, TimeUnit.SECONDS);
-    }
-
-    StreamConfigProvider getNullProvider() throws Exception {
-        return new NullStreamConfigProvider();
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceProviderWithConfigRouters() throws Exception {
-        getServiceProvider(new IdentityStreamPartitionConverter());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceProviderWithMissingConfig() throws Exception {
-        StreamConfigProvider provider = getServiceProvider(new IdentityStreamPartitionConverter());
-        Optional<DynamicDistributedLogConfiguration> config = provider.getDynamicStreamConfig("stream1");
-        assertTrue(config.isPresent());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceProviderWithDefaultConfigPath() throws Exception {
-        // Default config with property set.
-        PropertiesWriter writer1 = new PropertiesWriter();
-        writer1.setProperty("rpsStreamAcquireServiceLimit", "191919");
-        writer1.save();
-        String fallbackConfPath1 = writer1.getFile().getPath();
-        StreamConfigProvider provider1 = getServiceProvider(new IdentityStreamPartitionConverter(),
-                DEFAULT_CONFIG_DIR, fallbackConfPath1);
-        Optional<DynamicDistributedLogConfiguration> config1 = provider1.getDynamicStreamConfig("stream1");
-
-        // Empty default config.
-        PropertiesWriter writer2 = new PropertiesWriter();
-        writer2.save();
-        String fallbackConfPath2 = writer2.getFile().getPath();
-        StreamConfigProvider provider2 = getServiceProvider(new IdentityStreamPartitionConverter(),
-                DEFAULT_CONFIG_DIR, fallbackConfPath2);
-        Optional<DynamicDistributedLogConfiguration> config2 = provider2.getDynamicStreamConfig("stream1");
-
-        assertEquals(191919, config1.get().getRpsStreamAcquireServiceLimit());
-        assertEquals(-1, config2.get().getRpsStreamAcquireServiceLimit());
-    }
-
-    @Test(timeout = 60000)
-    public void testDefaultProvider() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, "99");
-        writer.save();
-        StreamConfigProvider provider = getDefaultProvider(writer.getFile().getPath());
-        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
-        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
-        assertTrue(config1.isPresent());
-        assertTrue(config1.get() == config2.get());
-        assertEquals(99, config1.get().getRetentionPeriodHours());
-    }
-
-    @Test(timeout = 60000)
-    public void testNullProvider() throws Exception {
-        StreamConfigProvider provider = getNullProvider();
-        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
-        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
-        assertFalse(config1.isPresent());
-        assertTrue(config1 == config2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
deleted file mode 100644
index 5f5ecd4..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.LinkedHashSet;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test Case for {@link LeastLoadPlacementPolicy}.
- */
-public class TestLeastLoadPlacementPolicy {
-
-    @Test(timeout = 10000)
-    public void testCalculateBalances() throws Exception {
-        int numSevers = new Random().nextInt(20) + 1;
-        int numStreams = new Random().nextInt(200) + 1;
-        RoutingService mockRoutingService = mock(RoutingService.class);
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
-        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
-            new EqualLoadAppraiser(),
-            mockRoutingService,
-            mockNamespace,
-            null,
-            Duration.fromSeconds(600),
-            new NullStatsLogger());
-        TreeSet<ServerLoad> serverLoads =
-            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
-        long lowLoadPerServer = numStreams / numSevers;
-        long highLoadPerServer = lowLoadPerServer + 1;
-        for (ServerLoad serverLoad : serverLoads) {
-            long load = serverLoad.getLoad();
-            assertEquals(load, serverLoad.getStreamLoads().size());
-            assertTrue(String.format("Load %d is not between %d and %d",
-                load, lowLoadPerServer, highLoadPerServer), load == lowLoadPerServer || load == highLoadPerServer);
-        }
-    }
-
-    @Test(timeout = 10000)
-    public void testRefreshAndPlaceStream() throws Exception {
-        int numSevers = new Random().nextInt(20) + 1;
-        int numStreams = new Random().nextInt(200) + 1;
-        RoutingService mockRoutingService = mock(RoutingService.class);
-        when(mockRoutingService.getHosts()).thenReturn(generateSocketAddresses(numSevers));
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
-        try {
-            when(mockNamespace.getLogs()).thenReturn(generateStreams(numStreams).iterator());
-        } catch (IOException e) {
-            fail();
-        }
-        PlacementStateManager mockPlacementStateManager = mock(PlacementStateManager.class);
-        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
-            new EqualLoadAppraiser(),
-            mockRoutingService,
-            mockNamespace,
-            mockPlacementStateManager,
-            Duration.fromSeconds(600),
-            new NullStatsLogger());
-        leastLoadPlacementPolicy.refresh();
-
-        final ArgumentCaptor<TreeSet> captor = ArgumentCaptor.forClass(TreeSet.class);
-        verify(mockPlacementStateManager).saveOwnership(captor.capture());
-        TreeSet<ServerLoad> serverLoads = (TreeSet<ServerLoad>) captor.getValue();
-        ServerLoad next = serverLoads.first();
-        String serverPlacement = Await.result(leastLoadPlacementPolicy.placeStream("newstream1"));
-        assertEquals(next.getServer(), serverPlacement);
-    }
-
-    @Test(timeout = 10000)
-    public void testCalculateUnequalWeight() throws Exception {
-        int numSevers = new Random().nextInt(20) + 1;
-        int numStreams = new Random().nextInt(200) + 1;
-    /* use AtomicInteger to have a final object in answer method */
-        final AtomicInteger maxLoad = new AtomicInteger(Integer.MIN_VALUE);
-        RoutingService mockRoutingService = mock(RoutingService.class);
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
-        LoadAppraiser mockLoadAppraiser = mock(LoadAppraiser.class);
-        when(mockLoadAppraiser.getStreamLoad(anyString())).then(new Answer<Future<StreamLoad>>() {
-            @Override
-            public Future<StreamLoad> answer(InvocationOnMock invocationOnMock) throws Throwable {
-                int load = new Random().nextInt(100000);
-                if (load > maxLoad.get()) {
-                    maxLoad.set(load);
-                }
-                return Future.value(new StreamLoad(invocationOnMock.getArguments()[0].toString(), load));
-            }
-        });
-        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
-            mockLoadAppraiser,
-            mockRoutingService,
-            mockNamespace,
-            null,
-            Duration.fromSeconds(600),
-            new NullStatsLogger());
-        TreeSet<ServerLoad> serverLoads =
-            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
-        long highestLoadSeen = Long.MIN_VALUE;
-        long lowestLoadSeen = Long.MAX_VALUE;
-        for (ServerLoad serverLoad : serverLoads) {
-            long load = serverLoad.getLoad();
-            if (load < lowestLoadSeen) {
-                lowestLoadSeen = load;
-            }
-            if (load > highestLoadSeen) {
-                highestLoadSeen = load;
-            }
-        }
-        assertTrue("Unexpected placement for " + numStreams + " streams to "
-                + numSevers + " servers : highest load = " + highestLoadSeen
-                + ", lowest load = " + lowestLoadSeen + ", max stream load = " + maxLoad.get(),
-            highestLoadSeen - lowestLoadSeen <= maxLoad.get());
-    }
-
-    private Set<SocketAddress> generateSocketAddresses(int num) {
-        LinkedHashSet<SocketAddress> socketAddresses = new LinkedHashSet<SocketAddress>();
-        for (int i = 0; i < num; i++) {
-            socketAddresses.add(new InetSocketAddress(i));
-        }
-        return socketAddresses;
-    }
-
-    private Set<String> generateStreams(int num) {
-        LinkedHashSet<String> streams = new LinkedHashSet<String>();
-        for (int i = 0; i < num; i++) {
-            streams.add("stream_" + i);
-        }
-        return streams;
-    }
-
-    private Set<String> generateServers(int num) {
-        LinkedHashSet<String> servers = new LinkedHashSet<String>();
-        for (int i = 0; i < num; i++) {
-            servers.add("server_" + i);
-        }
-        return servers;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java
deleted file mode 100644
index 5bd234f..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServerLoad}.
- */
-public class TestServerLoad {
-
-    @Test(timeout = 60000)
-    public void testSerializeDeserialize() throws IOException {
-        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
-        for (int i = 0; i < 20; i++) {
-            serverLoad.addStream(new StreamLoad("stream-" + i, i));
-        }
-        assertEquals(serverLoad, ServerLoad.deserialize(serverLoad.serialize()));
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLoad() throws IOException {
-        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
-        assertEquals(0, serverLoad.getLoad());
-        serverLoad.addStream(new StreamLoad("stream-" + 1, 3));
-        assertEquals(3, serverLoad.getLoad());
-        serverLoad.addStream(new StreamLoad("stream-" + 2, 7));
-        assertEquals(10, serverLoad.getLoad());
-        serverLoad.addStream(new StreamLoad("stream-" + 3, 1));
-        assertEquals(11, serverLoad.getLoad());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java
deleted file mode 100644
index 36a6fed..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StreamLoad}.
- */
-public class TestStreamLoad {
-
-    @Test(timeout = 10000)
-    public void testSerializeDeserialize() throws IOException {
-        final String streamName = "aHellaRandomStreamName";
-        final int load = 1337;
-        final StreamLoad streamLoad = new StreamLoad(streamName, load);
-        assertEquals(streamLoad, StreamLoad.deserialize(streamLoad.serialize()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
deleted file mode 100644
index 07ec5a5..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.placement;
-
-import static org.apache.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-import org.apache.distributedlog.DistributedLogConfiguration;
-import java.io.IOException;
-import java.net.URI;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.LinkedBlockingQueue;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.curator.test.TestingServer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ZKPlacementStateManager}.
- */
-public class TestZKPlacementStateManager {
-    private TestingServer zkTestServer;
-    private String zkServers;
-    private URI uri;
-    private ZKPlacementStateManager zkPlacementStateManager;
-
-    @Before
-    public void startZookeeper() throws Exception {
-        zkTestServer = new TestingServer(2181);
-        zkServers = "127.0.0.1:2181";
-        uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
-        zkPlacementStateManager =
-            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
-    }
-
-    @Test(timeout = 60000)
-    public void testSaveLoad() throws Exception {
-        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
-        zkPlacementStateManager.saveOwnership(ownerships);
-        SortedSet<ServerLoad> loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-
-        ownerships.add(new ServerLoad("emptyServer"));
-        zkPlacementStateManager.saveOwnership(ownerships);
-        loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-
-        ServerLoad sl1 = new ServerLoad("server1");
-        sl1.addStream(new StreamLoad("stream1", 3));
-        sl1.addStream(new StreamLoad("stream2", 4));
-        ServerLoad sl2 = new ServerLoad("server2");
-        sl2.addStream(new StreamLoad("stream3", 1));
-        ownerships.add(sl1);
-        ownerships.add(sl2);
-        zkPlacementStateManager.saveOwnership(ownerships);
-        loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-
-        loadedOwnerships.remove(sl1);
-        zkPlacementStateManager.saveOwnership(ownerships);
-        loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-    }
-
-    private TreeSet<ServerLoad> waitForServerLoadsNotificationAsc(
-        LinkedBlockingQueue<TreeSet<ServerLoad>> notificationQueue,
-        int expectedNumServerLoads) throws InterruptedException {
-        TreeSet<ServerLoad> notification = notificationQueue.take();
-        assertNotNull(notification);
-        while (notification.size() < expectedNumServerLoads) {
-            notification = notificationQueue.take();
-        }
-        assertEquals(expectedNumServerLoads, notification.size());
-        return notification;
-    }
-
-    @Test(timeout = 60000)
-    public void testWatchIndefinitely() throws Exception {
-        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
-        ownerships.add(new ServerLoad("server1"));
-        final LinkedBlockingQueue<TreeSet<ServerLoad>> serverLoadNotifications =
-            new LinkedBlockingQueue<TreeSet<ServerLoad>>();
-        PlacementStateManager.PlacementCallback callback = new PlacementStateManager.PlacementCallback() {
-            @Override
-            public void callback(TreeSet<ServerLoad> serverLoads) {
-                serverLoadNotifications.add(serverLoads);
-            }
-        };
-        zkPlacementStateManager.saveOwnership(ownerships); // need to initialize the zk path before watching
-        zkPlacementStateManager.watch(callback);
-        // cannot verify the callback here as it may call before the verify is called
-
-        zkPlacementStateManager.saveOwnership(ownerships);
-        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 1));
-
-        ServerLoad server2 = new ServerLoad("server2");
-        server2.addStream(new StreamLoad("hella-important-stream", 415));
-        ownerships.add(server2);
-        zkPlacementStateManager.saveOwnership(ownerships);
-        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 2));
-    }
-
-    @Test(timeout = 60000)
-    public void testZkFormatting() throws Exception {
-        final String server = "host/10.0.0.0:31351";
-        final String zkFormattedServer = "host--10.0.0.0:31351";
-        URI uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
-        ZKPlacementStateManager zkPlacementStateManager =
-            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
-        assertEquals(zkFormattedServer, zkPlacementStateManager.serverToZkFormat(server));
-        assertEquals(server, zkPlacementStateManager.zkFormatToServer(zkFormattedServer));
-    }
-
-    @After
-    public void stopZookeeper() throws IOException {
-        zkTestServer.stop();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
deleted file mode 100644
index 56e9483..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES 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.stream;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.service.config.StreamConfigProvider;
-import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.util.Await;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Case for StreamManager.
- */
-public class TestStreamManager {
-
-    @Rule
-    public TestName testName = new TestName();
-
-    ScheduledExecutorService mockExecutorService = mock(ScheduledExecutorService.class);
-
-    @Test(timeout = 60000)
-    public void testCollectionMethods() throws Exception {
-        Stream mockStream = mock(Stream.class);
-        when(mockStream.getStreamName()).thenReturn("stream1");
-        when(mockStream.getPartition()).thenReturn(new Partition("stream1", 0));
-        StreamFactory mockStreamFactory = mock(StreamFactory.class);
-        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
-        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
-        when(mockStreamFactory.create(
-                (String) any(),
-                (DynamicDistributedLogConfiguration) any(),
-                (StreamManager) any())).thenReturn(mockStream);
-        StreamManager streamManager = new StreamManagerImpl(
-                "",
-                new DistributedLogConfiguration(),
-                mockExecutorService,
-                mockStreamFactory,
-                mockPartitionConverter,
-                mockStreamConfigProvider,
-                mock(DistributedLogNamespace.class));
-
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyAcquired(mockStream);
-        assertTrue(streamManager.isAcquired("stream1"));
-        assertEquals(1, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyReleased(mockStream);
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyAcquired(mockStream);
-        assertTrue(streamManager.isAcquired("stream1"));
-        assertEquals(1, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyAcquired(mockStream);
-        assertTrue(streamManager.isAcquired("stream1"));
-        assertEquals(1, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyReleased(mockStream);
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyReleased(mockStream);
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateStream() throws Exception {
-        Stream mockStream = mock(Stream.class);
-        final String streamName = "stream1";
-        when(mockStream.getStreamName()).thenReturn(streamName);
-        StreamFactory mockStreamFactory = mock(StreamFactory.class);
-        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
-        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
-        when(mockStreamFactory.create(
-            (String) any(),
-            (DynamicDistributedLogConfiguration) any(),
-            (StreamManager) any())
-        ).thenReturn(mockStream);
-        DistributedLogNamespace dlNamespace = mock(DistributedLogNamespace.class);
-        ScheduledExecutorService executorService = new ScheduledThreadPoolExecutor(1);
-
-        StreamManager streamManager = new StreamManagerImpl(
-                "",
-                new DistributedLogConfiguration(),
-                executorService,
-                mockStreamFactory,
-                mockPartitionConverter,
-                mockStreamConfigProvider,
-                dlNamespace);
-
-        assertTrue(Await.ready(streamManager.createStreamAsync(streamName)).isReturn());
-        verify(dlNamespace).createLog(streamName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/c44e0278/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
deleted file mode 100644
index a18fda1..0000000
--- a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.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 org.apache.distributedlog.service.stream;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.distributedlog.AsyncLogWriter;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.LogRecord;
-import org.apache.distributedlog.acl.DefaultAccessControlManager;
-import org.apache.distributedlog.exceptions.InternalServerException;
-import org.apache.distributedlog.service.ResponseUtils;
-import org.apache.distributedlog.service.config.ServerConfiguration;
-import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import org.apache.distributedlog.thrift.service.StatusCode;
-import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import java.nio.ByteBuffer;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Case for StreamOps.
- */
-public class TestStreamOp {
-
-    @Rule
-    public TestName testName = new TestName();
-
-    private WriteOp getWriteOp() {
-        SettableFeature disabledFeature = new SettableFeature("", 0);
-        return new WriteOp("test",
-            ByteBuffer.wrap("test".getBytes()),
-            new NullStatsLogger(),
-            new NullStatsLogger(),
-            new IdentityStreamPartitionConverter(),
-            new ServerConfiguration(),
-            (byte) 0,
-            null,
-            false,
-            disabledFeature,
-            DefaultAccessControlManager.INSTANCE);
-    }
-
-    @Test(timeout = 60000)
-    public void testResponseFailedTwice() throws Exception {
-        WriteOp writeOp = getWriteOp();
-        writeOp.fail(new InternalServerException("test1"));
-        writeOp.fail(new InternalServerException("test2"));
-
-        WriteResponse response = Await.result(writeOp.result());
-        assertEquals(StatusCode.INTERNAL_SERVER_ERROR, response.getHeader().getCode());
-        assertEquals(ResponseUtils.exceptionToHeader(new InternalServerException("test1")), response.getHeader());
-    }
-
-    @Test(timeout = 60000)
-    public void testResponseSucceededThenFailed() throws Exception {
-        AsyncLogWriter writer = mock(AsyncLogWriter.class);
-        when(writer.write((LogRecord) any())).thenReturn(Future.value(new DLSN(1, 2, 3)));
-        when(writer.getStreamName()).thenReturn("test");
-        WriteOp writeOp = getWriteOp();
-        writeOp.execute(writer, new Sequencer() {
-            public long nextId() {
-                return 0;
-            }
-        }, new Object());
-        writeOp.fail(new InternalServerException("test2"));
-
-        WriteResponse response = Await.result(writeOp.result());
-        assertEquals(StatusCode.SUCCESS, response.getHeader().getCode());
-    }
-}