You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by ka...@apache.org on 2015/07/17 00:44:02 UTC
[2/8] storm git commit: make disruptor queue wait timeout configurable
make disruptor queue wait timeout configurable
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/68bd67c0
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/68bd67c0
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/68bd67c0
Branch: refs/heads/master
Commit: 68bd67c02af316e711fba98a37d853063e83884a
Parents: 5141440
Author: errordaiwa <xi...@outlook.com>
Authored: Fri Jul 10 11:00:28 2015 +0800
Committer: errordaiwa <xi...@outlook.com>
Committed: Fri Jul 10 11:00:28 2015 +0800
----------------------------------------------------------------------
conf/defaults.yaml | 1 +
storm-core/src/clj/backtype/storm/daemon/executor.clj | 1 +
storm-core/src/clj/backtype/storm/daemon/worker.clj | 2 ++
storm-core/src/clj/backtype/storm/disruptor.clj | 4 ++--
storm-core/src/jvm/backtype/storm/Config.java | 7 +++++++
storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java | 8 ++++++--
.../test/jvm/backtype/storm/utils/DisruptorQueueTest.java | 2 +-
7 files changed, 20 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/68bd67c0/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 875543f..47dc266 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -200,5 +200,6 @@ topology.testing.always.try.serialize: false
topology.classpath: null
topology.environment: null
topology.bolts.outgoing.overflow.buffer.enable: false
+topology.disruptor.wait.timeout.millis: 10
dev.zookeeper.path: "/tmp/dev-storm-zookeeper"
http://git-wip-us.apache.org/repos/asf/storm/blob/68bd67c0/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index 454fd0d..75414d0 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -223,6 +223,7 @@
batch-transfer->worker (disruptor/disruptor-queue
(str "executor" executor-id "-send-queue")
(storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
+ (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
:claim-strategy :single-threaded
:wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
]
http://git-wip-us.apache.org/repos/asf/storm/blob/68bd67c0/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 7157cf7..2e64fb4 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -153,6 +153,7 @@
;; TODO: this depends on the type of executor
(map (fn [e] [e (disruptor/disruptor-queue (str "receive-queue" e)
(storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
+ (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
:wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))]))
(into {})
))
@@ -194,6 +195,7 @@
(let [assignment-versions (atom {})
executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
+ (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
:wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
http://git-wip-us.apache.org/repos/asf/storm/blob/68bd67c0/storm-core/src/clj/backtype/storm/disruptor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/disruptor.clj b/storm-core/src/clj/backtype/storm/disruptor.clj
index a723601..e96e49d 100644
--- a/storm-core/src/clj/backtype/storm/disruptor.clj
+++ b/storm-core/src/clj/backtype/storm/disruptor.clj
@@ -45,10 +45,10 @@
;; wouldn't make it to the acker until the batch timed out and another tuple was played into the queue,
;; unblocking the consumer
(defnk disruptor-queue
- [^String queue-name buffer-size :claim-strategy :multi-threaded :wait-strategy :block]
+ [^String queue-name buffer-size timeout :claim-strategy :multi-threaded :wait-strategy :block]
(DisruptorQueue. queue-name
((CLAIM-STRATEGY claim-strategy) buffer-size)
- (mk-wait-strategy wait-strategy)))
+ (mk-wait-strategy wait-strategy) timeout))
(defn clojure-handler
[afn]
http://git-wip-us.apache.org/repos/asf/storm/blob/68bd67c0/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 063d446..f20f281 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1452,6 +1452,13 @@ public class Config extends HashMap<String, Object> {
public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class;
+ /**
+ * Configure timeout milliseconds used for disruptor queue wait strategy. Can be used to tradeoff latency
+ * vs. CPU usage
+ */
+ public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis";
+ public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = Number.class;
+
public static void setClasspath(Map conf, String cp) {
conf.put(Config.TOPOLOGY_CLASSPATH, cp);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/68bd67c0/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
index 1f2110d..195c2f7 100644
--- a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
@@ -60,8 +60,10 @@ public class DisruptorQueue implements IStatefulObject {
private static String PREFIX = "disruptor-";
private String _queueName = "";
+
+ private long _waitTimeout;
- public DisruptorQueue(String queueName, ClaimStrategy claim, WaitStrategy wait) {
+ public DisruptorQueue(String queueName, ClaimStrategy claim, WaitStrategy wait, long timeout) {
this._queueName = PREFIX + queueName;
_buffer = new RingBuffer<MutableObject>(new ObjectEventFactory(), claim, wait);
_consumer = new Sequence();
@@ -77,6 +79,8 @@ public class DisruptorQueue implements IStatefulObject {
throw new RuntimeException("This code should be unreachable!", e);
}
}
+
+ _waitTimeout = timeout;
}
public String getName() {
@@ -94,7 +98,7 @@ public class DisruptorQueue implements IStatefulObject {
public void consumeBatchWhenAvailable(EventHandler<Object> handler) {
try {
final long nextSequence = _consumer.get() + 1;
- final long availableSequence = _barrier.waitFor(nextSequence, 1000, TimeUnit.MILLISECONDS);
+ final long availableSequence = _barrier.waitFor(nextSequence, _waitTimeout, TimeUnit.MILLISECONDS);
if(availableSequence >= nextSequence) {
consumeBatchToCursor(availableSequence, handler);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/68bd67c0/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java
index 61af50d..ddc0982 100644
--- a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java
+++ b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java
@@ -154,6 +154,6 @@ public class DisruptorQueueTest extends TestCase {
private static DisruptorQueue createQueue(String name, int queueSize) {
return new DisruptorQueue(name, new MultiThreadedClaimStrategy(
- queueSize), new BlockingWaitStrategy());
+ queueSize), new BlockingWaitStrategy(), 10L);
}
}