You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2014/06/09 15:48:07 UTC

[02/32] git commit: STORM-297, add thread factory and TransferDrainer

STORM-297, add thread factory and TransferDrainer


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

Branch: refs/heads/master
Commit: be8f8327b3c2d36e89420d62fb8ca530ba726cf3
Parents: 861a92e
Author: Sean Zhong <cl...@gmail.com>
Authored: Wed May 7 11:17:53 2014 +0800
Committer: Sean Zhong <cl...@gmail.com>
Committed: Wed May 7 11:17:53 2014 +0800

----------------------------------------------------------------------
 .../netty/NettyRenameThreadFactory.java         |  35 ++++++
 .../backtype/storm/utils/TransferDrainer.java   | 113 +++++++++++++++++++
 2 files changed, 148 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/be8f8327/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
new file mode 100644
index 0000000..ea3f249
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
@@ -0,0 +1,35 @@
+package backtype.storm.messaging.netty;
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jboss.netty.util.ThreadNameDeterminer;
+import org.jboss.netty.util.ThreadRenamingRunnable;
+
+public class NettyRenameThreadFactory  implements ThreadFactory {
+    
+    static {
+      //Rename Netty threads
+      ThreadRenamingRunnable.setThreadNameDeterminer(ThreadNameDeterminer.CURRENT);
+    }
+  
+    final ThreadGroup group;
+    final AtomicInteger index = new AtomicInteger(1);
+    final String name;
+
+    NettyRenameThreadFactory(String name) {
+        SecurityManager s = System.getSecurityManager();
+        group = (s != null)? s.getThreadGroup() :
+                             Thread.currentThread().getThreadGroup();
+        this.name = name;
+    }
+
+    public Thread newThread(Runnable r) {
+        Thread t = new Thread(group, r, name + "-" + index.getAndIncrement(), 0);
+        if (t.isDaemon())
+            t.setDaemon(false);
+        if (t.getPriority() != Thread.NORM_PRIORITY)
+            t.setPriority(Thread.NORM_PRIORITY);
+        return t;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/be8f8327/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java b/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
new file mode 100644
index 0000000..c9f1b04
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import backtype.storm.messaging.IConnection;
+import backtype.storm.messaging.TaskMessage;
+
+public class TransferDrainer {
+
+  private HashMap<String, ArrayList<ArrayList<TaskMessage>>> bundles = new HashMap();
+  
+  public void add(HashMap<String, ArrayList<TaskMessage>> workerTupleSetMap) {
+    for (String key : workerTupleSetMap.keySet()) {
+      
+      ArrayList<ArrayList<TaskMessage>> bundle = bundles.get(key);
+      if (null == bundle) {
+        bundle = new ArrayList<ArrayList<TaskMessage>>();
+        bundles.put(key, bundle);
+      }
+      
+      ArrayList tupleSet = workerTupleSetMap.get(key);
+      if (null != tupleSet && tupleSet.size() > 0) {
+        bundle.add(tupleSet);
+      }
+    } 
+  }
+  
+  public void send(HashMap<String, IConnection> connections) {
+    for (String hostPort : bundles.keySet()) {
+      IConnection connection = connections.get(hostPort);
+      if (null != connection) { 
+        ArrayList<ArrayList<TaskMessage>> bundle = bundles.get(hostPort);
+        Iterator<TaskMessage> iter = getBundleIterator(bundle);
+        if (null != iter) {
+          connection.send(iter);
+        }
+      }
+    } 
+  }
+  
+  private Iterator<TaskMessage> getBundleIterator(final ArrayList<ArrayList<TaskMessage>> bundle) {
+    
+    if (null == bundle) {
+      return null;
+    }
+    
+    return new Iterator<TaskMessage> () {
+      
+      private int offset = 0;
+      private int size = 0;
+      {
+        for (ArrayList<TaskMessage> list : bundle) {
+            size += list.size();
+        }
+      }
+      
+      private int bundleOffset = 0;
+      private Iterator<TaskMessage> iter = bundle.get(bundleOffset).iterator();
+      
+      @Override
+      public boolean hasNext() {
+        if (offset < size) {
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      public TaskMessage next() {
+        TaskMessage msg = null;
+        if (iter.hasNext()) {
+          msg = iter.next(); 
+        } else {
+          bundleOffset++;
+          iter = bundle.get(bundleOffset).iterator();
+          msg = iter.next();
+        }
+        if (null != msg) {
+          offset++;
+        }
+        return msg;
+      }
+
+      @Override
+      public void remove() {
+        throw new RuntimeException("not supported");
+      }
+    };
+  }
+  
+  public void clear() {
+    bundles.clear();
+  }
+}
\ No newline at end of file