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