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:17 UTC

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

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;
-    }
-}