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/05/19 23:52:26 UTC

[01/24] git commit: Shell component serialisation is now plugable with a default JSON serialiser being used. Moved multilang classes to multilang directory.

Repository: incubator-storm
Updated Branches:
  refs/heads/master 1311a1ddf -> a09adda06


Shell component serialisation is now plugable with a default JSON serialiser being used. Moved multilang classes to multilang directory.


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

Branch: refs/heads/master
Commit: 1a19f3ddaca0d2ab2c388d70d03d7ebd63590879
Parents: 504b552
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Tue Oct 1 10:59:47 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Tue Oct 1 10:59:47 2013 +0200

----------------------------------------------------------------------
 project.clj                                     |   2 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj |   5 +-
 .../backtype/storm/clojure/RichShellBolt.java   |   2 +-
 .../backtype/storm/clojure/RichShellSpout.java  |   2 +-
 .../jvm/backtype/storm/multilang/Emission.java  |  69 ++++++
 .../backtype/storm/multilang/ISerializer.java   |  19 ++
 .../jvm/backtype/storm/multilang/Immission.java |  42 ++++
 .../storm/multilang/JsonSerializer.java         | 144 ++++++++++++
 .../storm/multilang/NoOutputException.java      |   8 +
 .../jvm/backtype/storm/multilang/ShellBolt.java | 207 +++++++++++++++++
 .../backtype/storm/multilang/ShellProcess.java  | 101 +++++++++
 .../backtype/storm/multilang/ShellSpout.java    | 115 ++++++++++
 .../jvm/backtype/storm/multilang/SpoutMsg.java  |  18 ++
 .../jvm/backtype/storm/spout/ShellSpout.java    | 120 ----------
 .../src/jvm/backtype/storm/task/ShellBolt.java  | 223 -------------------
 .../jvm/backtype/storm/utils/ShellProcess.java  | 132 -----------
 16 files changed, 729 insertions(+), 480 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/project.clj
----------------------------------------------------------------------
diff --git a/project.clj b/project.clj
index ed1fdbc..f913924 100644
--- a/project.clj
+++ b/project.clj
@@ -10,7 +10,7 @@
                  :archive "https://groups.google.com/group/storm-user"
                  :post "storm-user@googlegroups.com"}
   :dependencies [~@DEPENDENCIES]
-  :plugins [[~'lein-sub "0.2.1"]]  
+  :plugins [[~'lein-sub "0.2.1"],[no-man-is-an-island/lein-eclipse "2.0.0"]]
   :min-lein-version "2.0.0"
   :sub [~@MODULES]
   ))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/clj/backtype/storm/bootstrap.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index 70b8005..1075c66 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -12,10 +12,11 @@
                      RegisteredGlobalState ThriftTopologyUtils DisruptorQueue
                      MutableObject MutableLong]))
      (import (quote [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer]))
-     (import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector ShellSpout]))
+     (import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector]))
+     (import (quote [backtype.storm.multilang ShellSpout ShellBolt]))
      (import (quote [backtype.storm.tuple Tuple TupleImpl Fields MessageId]))
      (import (quote [backtype.storm.task IBolt IOutputCollector
-                     OutputCollector TopologyContext ShellBolt
+                     OutputCollector TopologyContext
                      GeneralTopologyContext WorkerTopologyContext]))
      (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs 
                      IBatchBolt BatchBoltExecutor]))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java b/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
index 6be104e..6ff669e 100644
--- a/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
@@ -1,7 +1,7 @@
 package backtype.storm.clojure;
 
 import backtype.storm.generated.StreamInfo;
-import backtype.storm.task.ShellBolt;
+import backtype.storm.multilang.ShellBolt;
 import backtype.storm.topology.IRichBolt;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java b/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
index cb5947f..57410e1 100644
--- a/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
@@ -1,7 +1,7 @@
 package backtype.storm.clojure;
 
 import backtype.storm.generated.StreamInfo;
-import backtype.storm.spout.ShellSpout;
+import backtype.storm.multilang.ShellSpout;
 import backtype.storm.topology.IRichSpout;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/Emission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Emission.java b/storm-core/src/jvm/backtype/storm/multilang/Emission.java
new file mode 100644
index 0000000..1054458
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/Emission.java
@@ -0,0 +1,69 @@
+package backtype.storm.multilang;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Emission {
+	private String command;
+	private String id;
+	private List<String> anchors;
+	private String stream;
+	private long task;
+	private String msg;
+	private List<Object> tuple;
+	
+	public String getCommand() {
+		return command;
+	}
+	public void setCommand(String command) {
+		this.command = command;
+	}
+	public String getId() {
+		return id;
+	}
+	public void setId(String id) {
+		this.id = id;
+	}
+	public List<String> getAnchors() {
+		return anchors;
+	}
+	public void setAnchors(List<String> anchors) {
+		this.anchors = anchors;
+	}
+	public void addAnchor(String anchor) {
+		if (anchors == null) {
+			anchors = new ArrayList<String>();
+		}
+		this.anchors.add(anchor);
+	}
+	public String getStream() {
+		return stream;
+	}
+	public void setStream(String stream) {
+		this.stream = stream;
+	}
+	public long getTask() {
+		return task;
+	}
+	public void setTask(long task) {
+		this.task = task;
+	}
+	public String getMsg() {
+		return msg;
+	}
+	public void setMsg(String msg) {
+		this.msg = msg;
+	}
+	public List<Object> getTuple() {
+		return tuple;
+	}
+	public void setTuple(List<Object> tuple) {
+		this.tuple = tuple;
+	}
+	public void addTuple(Object tuple) {
+		if (this.tuple == null) {
+			this.tuple = new ArrayList<Object>();
+		}
+		this.tuple.add(tuple);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
new file mode 100644
index 0000000..61a28a9
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
@@ -0,0 +1,19 @@
+package backtype.storm.multilang;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+import backtype.storm.task.TopologyContext;
+
+public interface ISerializer extends Serializable {
+	void initialize (OutputStream processIn, InputStream processOut);
+	Number connect (Map conf, TopologyContext context) throws IOException, NoOutputException;
+	Emission readEmission() throws IOException, NoOutputException;
+	void writeImmission(Immission immission) throws IOException;
+	void writeSpoutMsg(SpoutMsg msg) throws IOException;
+	void writeTaskIds(List<Integer> taskIds) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/Immission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Immission.java b/storm-core/src/jvm/backtype/storm/multilang/Immission.java
new file mode 100644
index 0000000..c20df53
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/Immission.java
@@ -0,0 +1,42 @@
+package backtype.storm.multilang;
+
+import java.util.List;
+
+public class Immission {
+	private String id;
+	private String comp;
+	private String stream;
+	private long task;
+	private List<Object> tuple;
+	
+	public String getId() {
+		return id;
+	}
+	public void setId(String id) {
+		this.id = id;
+	}
+	public String getComp() {
+		return comp;
+	}
+	public void setComp(String comp) {
+		this.comp = comp;
+	}
+	public String getStream() {
+		return stream;
+	}
+	public void setStream(String stream) {
+		this.stream = stream;
+	}
+	public long getTask() {
+		return task;
+	}
+	public void setTask(long task) {
+		this.task = task;
+	}
+	public List<Object> getTuple() {
+		return tuple;
+	}
+	public void setTuple(List<Object> tuple) {
+		this.tuple = tuple;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
new file mode 100644
index 0000000..88eead2
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -0,0 +1,144 @@
+package backtype.storm.multilang;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.json.simple.JSONObject;
+import org.json.simple.JSONValue;
+
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.Utils;
+
+public class JsonSerializer implements ISerializer {
+	private DataOutputStream processIn;
+    private BufferedReader processOut;
+
+	public void initialize (OutputStream processIn, InputStream processOut) {
+		this.processIn = new DataOutputStream(processIn);;
+		this.processOut = new BufferedReader(new InputStreamReader(processOut));
+	}
+
+	public Number connect (Map conf, TopologyContext context) throws IOException, NoOutputException {
+		JSONObject setupInfo = new JSONObject();
+        setupInfo.put("pidDir", context.getPIDDir());
+        setupInfo.put("conf", conf);
+        setupInfo.put("context", context);
+    	writeMessage(setupInfo);
+
+    	Number pid = (Number)((JSONObject)readMessage()).get("pid");
+        return pid;
+	}
+
+	public void writeImmission(Immission immission) throws IOException {
+		JSONObject obj = new JSONObject();
+        obj.put("id", immission.getId());
+        obj.put("comp", immission.getComp());
+        obj.put("stream", immission.getStream());
+        obj.put("task", immission.getTask());
+        obj.put("tuple", immission.getTuple());
+        writeMessage(obj);
+	}
+
+	public void writeSpoutMsg(SpoutMsg msg) throws IOException {
+		JSONObject obj = new JSONObject();
+        obj.put("command", msg.getCommand());
+        obj.put("id", msg.getId());
+        writeMessage(obj);
+	}
+
+	public void writeTaskIds(List<Integer> taskIds) throws IOException {
+        writeMessage(taskIds);
+	}
+
+	private void writeMessage(Object msg) throws IOException {
+        writeString(JSONValue.toJSONString(msg));
+    }
+
+	private void writeString(String str) throws IOException {
+        byte[] strBytes = str.getBytes("UTF-8");
+        processIn.write(strBytes, 0, strBytes.length);
+        processIn.writeBytes("\nend\n");
+        processIn.flush();
+    }
+
+	public Emission readEmission() throws IOException, NoOutputException {
+        JSONObject msg = (JSONObject)readMessage();
+    	Emission emission = new Emission();
+
+    	String command = (String) msg.get("command");
+        emission.setCommand(command);
+
+    	String stream = (String) msg.get("stream");
+        if(stream == null) stream = Utils.DEFAULT_STREAM_ID;
+        emission.setStream(stream);
+
+        Object taskObj = msg.get("task");
+        if (taskObj != null) {
+        	emission.setTask((Long) taskObj);
+        }
+
+        emission.setTuple((List) msg.get("tuple"));
+
+        List<Tuple> anchors = new ArrayList<Tuple>();
+        Object anchorObj = msg.get("anchors");
+        if(anchorObj!=null) {
+            if(anchorObj instanceof String) {
+                anchorObj = Arrays.asList(anchorObj);
+            }
+            for(Object o: (List) anchorObj) {
+                emission.addAnchor((String)o);
+            }
+        }
+
+        return emission;
+    }
+
+	private Object readMessage() throws IOException, NoOutputException {
+		String string = readString();
+        Object msg = JSONValue.parse(string);
+        if (msg != null) {
+        	return msg;
+        } else {
+        	throw new IOException("unable to parse: " + string);
+        }
+	}
+
+    private String readString() throws IOException, NoOutputException {
+        StringBuilder line = new StringBuilder();
+
+        //synchronized (processOut) {
+            while (true) {
+                String subline = processOut.readLine();
+                if(subline==null) {
+                    StringBuilder errorMessage = new StringBuilder();
+                    errorMessage.append("Pipe to subprocess seems to be broken!");
+                    if (line.length() == 0) {
+                        errorMessage.append(" No output read.\n");
+                    }
+                    else {
+                        errorMessage.append(" Currently read output: " + line.toString() + "\n");
+                    }
+                    errorMessage.append("Serializer Exception:\n");
+                    throw new NoOutputException(errorMessage.toString());
+                }
+                if(subline.equals("end")) {
+                    break;
+                }
+                if(line.length()!=0) {
+                    line.append("\n");
+                }
+                line.append(subline);
+            }
+        //}
+        return line.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
new file mode 100644
index 0000000..595a0ae
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
@@ -0,0 +1,8 @@
+package backtype.storm.multilang;
+
+public class NoOutputException extends Exception {
+	public NoOutputException() { super(); }
+	public NoOutputException(String message) { super(message); }
+	public NoOutputException(String message, Throwable cause) { super(message, cause); }
+	public NoOutputException(Throwable cause) { super(cause); }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java b/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java
new file mode 100644
index 0000000..acd60c9
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java
@@ -0,0 +1,207 @@
+package backtype.storm.multilang;
+
+import backtype.storm.generated.ShellComponent;
+import backtype.storm.task.IBolt;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.multilang.ShellProcess;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import java.util.Map;
+import java.util.Random;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A bolt that shells out to another process to process tuples. ShellBolt
+ * communicates with that process over stdio using a special protocol. An ~100
+ * line library is required to implement that protocol, and adapter libraries
+ * currently exist for Ruby and Python.
+ *
+ * <p>To run a ShellBolt on a cluster, the scripts that are shelled out to must be
+ * in the resources directory within the jar submitted to the master.
+ * During development/testing on a local machine, that resources directory just
+ * needs to be on the classpath.</p>
+ *
+ * <p>When creating topologies using the Java API, subclass this bolt and implement
+ * the IRichBolt interface to create components for the topology that use other languages. For example:
+ * </p>
+ *
+ * <pre>
+ * public class MyBolt extends ShellBolt implements IRichBolt {
+ *      public MyBolt() {
+ *          super("python", "mybolt.py");
+ *      }
+ *
+ *      public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ *          declarer.declare(new Fields("field1", "field2"));
+ *      }
+ * }
+ * </pre>
+ */
+public class ShellBolt implements IBolt {
+    public static Logger LOG = LoggerFactory.getLogger(ShellBolt.class);
+    Process _subprocess;
+    OutputCollector _collector;
+    Map<String, Tuple> _inputs = new ConcurrentHashMap<String, Tuple>();
+
+    private String[] _command;
+    private ShellProcess _process;
+    private volatile boolean _running = true;
+    private volatile Throwable _exception;
+    private LinkedBlockingQueue _pendingWrites = new LinkedBlockingQueue();
+    private Random _rand;
+
+    private Thread _readerThread;
+    private Thread _writerThread;
+
+    public ShellBolt(ShellComponent component) {
+        this(component.get_execution_command(), component.get_script());
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellBolt(String... command) {
+        _command = command;
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellBolt(ISerializer serializer, String... command) {
+        _command = command;
+        _process = new ShellProcess(serializer, _command);
+    }
+
+    public void prepare(Map stormConf, TopologyContext context,
+                        final OutputCollector collector) {
+        _rand = new Random();
+        _collector = collector;
+
+        //subprocesses must send their pid first thing
+        Number subpid = _process.launch(stormConf, context);
+        LOG.info("Launched subprocess with pid " + subpid);
+
+        // reader
+        _readerThread = new Thread(new Runnable() {
+            public void run() {
+                while (_running) {
+                    try {
+                        Emission emission = _process.readEmission();
+
+                        String command = emission.getCommand();
+                        if(command.equals("ack")) {
+                            handleAck(emission.getId());
+                        } else if (command.equals("fail")) {
+                            handleFail(emission.getId());
+                        } else if (command.equals("error")) {
+                            handleError(emission.getMsg());
+                        } else if (command.equals("log")) {
+                            String msg = emission.getMsg();
+                            LOG.info("Shell msg: " + msg);
+                        } else if (command.equals("emit")) {
+                            handleEmit(emission);
+                        }
+                    } catch (InterruptedException e) {
+                    } catch (Throwable t) {
+                        die(t);
+                    }
+                }
+            }
+        });
+
+        _readerThread.start();
+
+        _writerThread = new Thread(new Runnable() {
+            public void run() {
+                while (_running) {
+                    try {
+                    	// FIXME: This can either be TaskIds or Immissions
+                        Object write = _pendingWrites.poll(1, SECONDS);
+                        if (write instanceof Immission) {
+                            _process.writeImmission((Immission)write);
+                        } else if (write instanceof List<?>) {
+
+                        }
+                    } catch (InterruptedException e) {
+                    } catch (Throwable t) {
+                        die(t);
+                    }
+                }
+            }
+        });
+
+        _writerThread.start();
+    }
+
+    public void execute(Tuple input) {
+        if (_exception != null) {
+            throw new RuntimeException(_exception);
+        }
+
+        //just need an id
+        String genId = Long.toString(_rand.nextLong());
+        _inputs.put(genId, input);
+        try {
+            Immission immission = new Immission();
+            immission.setId(genId);
+            immission.setComp(input.getSourceComponent());
+            immission.setStream(input.getSourceStreamId());
+            immission.setTask(input.getSourceTask());
+            immission.setTuple(input.getValues());
+
+            _pendingWrites.put(immission);
+        } catch(InterruptedException e) {
+            throw new RuntimeException("Error during multilang processing", e);
+        }
+    }
+
+    public void cleanup() {
+        _running = false;
+        _process.destroy();
+        _inputs.clear();
+    }
+
+    private void handleAck(String id) {
+        Tuple acked = _inputs.remove(id);
+        if(acked==null) {
+            throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id);
+        }
+        _collector.ack(acked);
+    }
+
+    private void handleFail(String id) {
+        Tuple failed = _inputs.remove(id);
+        if(failed==null) {
+            throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id);
+        }
+        _collector.fail(failed);
+    }
+
+    private void handleError(String msg) {
+        _collector.reportError(new Exception("Shell Process Exception: " + msg));
+    }
+
+    private void handleEmit(Emission emission) throws InterruptedException {
+    	List<Tuple> anchors = new ArrayList<Tuple>();
+    	for (String anchor : emission.getAnchors()) {
+	    	Tuple t = _inputs.get(anchor);
+	        if (t == null) {
+	            throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
+	        }
+	        anchors.add(t);
+    	}
+
+        if(emission.getTask() == 0) {
+            List<Integer> outtasks = _collector.emit(emission.getStream(), anchors, emission.getTuple());
+            _pendingWrites.put(outtasks);
+        } else {
+            _collector.emitDirect((int)emission.getTask(), emission.getStream(), anchors, emission.getTuple());
+        }
+    }
+
+    private void die(Throwable exception) {
+        _exception = exception;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java b/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java
new file mode 100644
index 0000000..12689d8
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java
@@ -0,0 +1,101 @@
+package backtype.storm.multilang;
+
+import backtype.storm.task.TopologyContext;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.log4j.Logger;
+
+public class ShellProcess implements Serializable {
+    public static Logger LOG = Logger.getLogger(ShellProcess.class);
+    private Process _subprocess;
+    private InputStream processErrorStream;
+    private String[] command;
+    public ISerializer serializer;
+
+    public ShellProcess(ISerializer serializer, String[] command) {
+        this.command = command;
+        this.serializer = serializer;
+    }
+
+    public Number launch(Map conf, TopologyContext context) {
+        ProcessBuilder builder = new ProcessBuilder(command);
+        builder.directory(new File(context.getCodeDir()));
+
+        Number pid;
+        try {
+	        _subprocess = builder.start();
+	        processErrorStream = _subprocess.getErrorStream();
+	        serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream());
+	        pid = serializer.connect(conf, context);
+        } catch (IOException e) {
+        	throw new RuntimeException("Error when launching multilang subprocess\n" + getErrorsString(), e);
+        } catch (NoOutputException e) {
+            throw new RuntimeException(e + getErrorsString() + "\n");
+    	}
+        return pid;
+    }
+
+    public void destroy() {
+        _subprocess.destroy();
+    }
+
+    public Emission readEmission() throws IOException {
+    	try  {
+    		return serializer.readEmission();
+    	} catch (NoOutputException e) {
+            throw new RuntimeException(e + getErrorsString() + "\n");
+    	}
+    }
+
+    public void writeImmission(Immission msg) throws IOException {
+    	serializer.writeImmission(msg);
+        // drain the error stream to avoid dead lock because of full error stream buffer
+        drainErrorStream();
+    }
+
+    public void writeSpoutMsg(SpoutMsg msg) throws IOException {
+    	serializer.writeSpoutMsg(msg);
+        // drain the error stream to avoid dead lock because of full error stream buffer
+        drainErrorStream();
+    }
+
+    public void writeTaskIds(List<Integer> taskIds) throws IOException {
+    	serializer.writeTaskIds(taskIds);
+        // drain the error stream to avoid dead lock because of full error stream buffer
+        drainErrorStream();
+    }
+
+    public void drainErrorStream()
+    {
+        try {
+            while (processErrorStream.available() > 0)
+            {
+                int bufferSize = processErrorStream.available();
+                byte[] errorReadingBuffer =  new byte[bufferSize];
+
+                processErrorStream.read(errorReadingBuffer, 0, bufferSize);
+
+                LOG.info("Got error from shell process: " + new String(errorReadingBuffer));
+            }
+        } catch(Exception e) {
+        }
+    }
+
+    public String getErrorsString() {
+        if(processErrorStream!=null) {
+            try {
+                return IOUtils.toString(processErrorStream);
+            } catch(IOException e) {
+                return "(Unable to capture error stream)";
+            }
+        } else {
+            return "";
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java b/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java
new file mode 100644
index 0000000..7895a00
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java
@@ -0,0 +1,115 @@
+package backtype.storm.multilang;
+
+import backtype.storm.generated.ShellComponent;
+import backtype.storm.spout.ISpout;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.multilang.ShellProcess;
+import backtype.storm.utils.Utils;
+import java.util.Map;
+import java.util.List;
+import java.io.IOException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ShellSpout implements ISpout {
+    public static Logger LOG = LoggerFactory.getLogger(ShellSpout.class);
+
+    private SpoutOutputCollector _collector;
+    private String[] _command;
+    private ShellProcess _process;
+    private SpoutMsg spoutMsg;
+
+    public ShellSpout(ShellComponent component) {
+        this(component.get_execution_command(), component.get_script());
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellSpout(String... command) {
+        _command = command;
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellSpout(ISerializer serializer, String... command) {
+        _command = command;
+        _process = new ShellProcess(serializer, _command);
+    }
+
+    public void open(Map stormConf, TopologyContext context,
+                     SpoutOutputCollector collector) {
+        _collector = collector;
+
+        Number subpid = _process.launch(stormConf, context);
+        LOG.info("Launched subprocess with pid " + subpid);
+    }
+
+    public void close() {
+        _process.destroy();
+    }
+
+    public void nextTuple() {
+        if (spoutMsg == null) {
+            spoutMsg = new SpoutMsg();
+        }
+        spoutMsg.setCommand("next");
+        spoutMsg.setId("");
+        querySubprocess();
+    }
+
+    public void ack(Object msgId) {
+    	if (spoutMsg == null) {
+            spoutMsg = new SpoutMsg();
+        }
+        spoutMsg.setCommand("ack");
+        spoutMsg.setId(msgId.toString());
+        querySubprocess();
+    }
+
+    public void fail(Object msgId) {
+    	if (spoutMsg == null) {
+            spoutMsg = new SpoutMsg();
+        }
+        spoutMsg.setCommand("fail");
+        spoutMsg.setId(msgId.toString());
+        querySubprocess();
+    }
+
+    private void querySubprocess() {
+        try {
+            _process.writeSpoutMsg(spoutMsg);
+
+            while (true) {
+                Emission emission = _process.readEmission();
+                String command = emission.getCommand();
+                if (command.equals("sync")) {
+                    return;
+                } else if (command.equals("log")) {
+                    String msg = emission.getMsg();
+                    LOG.info("Shell msg: " + msg);
+                } else if (command.equals("emit")) {
+                    String stream = emission.getStream();
+                    Long task = emission.getTask();
+                    List<Object> tuple = emission.getTuple();
+                    Object messageId = emission.getId();
+                    if (task == 0) {
+                        List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
+                        _process.writeTaskIds(outtasks);
+                    } else {
+                        _collector.emitDirect((int)task.longValue(), stream, tuple, messageId);
+                    }
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void activate() {
+    }
+
+    @Override
+    public void deactivate() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
new file mode 100644
index 0000000..00967cb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
@@ -0,0 +1,18 @@
+package backtype.storm.multilang;
+
+public class SpoutMsg {
+	private String command;
+	private String id;
+	public String getCommand() {
+		return command;
+	}
+	public void setCommand(String command) {
+		this.command = command;
+	}
+	public String getId() {
+		return id;
+	}
+	public void setId(String id) {
+		this.id = id;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
deleted file mode 100644
index ba7bb63..0000000
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ /dev/null
@@ -1,120 +0,0 @@
-package backtype.storm.spout;
-
-import backtype.storm.generated.ShellComponent;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.utils.ShellProcess;
-import backtype.storm.utils.Utils;
-import java.util.Map;
-import java.util.List;
-import java.io.IOException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.json.simple.JSONObject;
-
-
-public class ShellSpout implements ISpout {
-    public static Logger LOG = LoggerFactory.getLogger(ShellSpout.class);
-
-    private SpoutOutputCollector _collector;
-    private String[] _command;
-    private ShellProcess _process;
-
-    public ShellSpout(ShellComponent component) {
-        this(component.get_execution_command(), component.get_script());
-    }
-    
-    public ShellSpout(String... command) {
-        _command = command;
-    }
-    
-    public void open(Map stormConf, TopologyContext context,
-                     SpoutOutputCollector collector) {
-        _process = new ShellProcess(_command);
-        _collector = collector;
-
-        try {
-            Number subpid = _process.launch(stormConf, context);
-            LOG.info("Launched subprocess with pid " + subpid);
-        } catch (IOException e) {
-            throw new RuntimeException("Error when launching multilang subprocess\n" + _process.getErrorsString(), e);
-        }
-    }
-
-    public void close() {
-        _process.destroy();
-    }
-
-    private JSONObject _next;
-    public void nextTuple() {
-        if (_next == null) {
-            _next = new JSONObject();
-            _next.put("command", "next");
-        }
-
-        querySubprocess(_next);
-    }
-
-    private JSONObject _ack;
-    public void ack(Object msgId) {
-        if (_ack == null) {
-            _ack = new JSONObject();
-            _ack.put("command", "ack");
-        }
-
-        _ack.put("id", msgId);
-        querySubprocess(_ack);
-    }
-
-    private JSONObject _fail;
-    public void fail(Object msgId) {
-        if (_fail == null) {
-            _fail = new JSONObject();
-            _fail.put("command", "fail");
-        }
-
-        _fail.put("id", msgId);
-        querySubprocess(_fail);
-    }
-
-    private void querySubprocess(Object query) {
-        try {
-            _process.writeMessage(query);
-
-            while (true) {
-                JSONObject action = _process.readMessage();
-                String command = (String) action.get("command");
-                if (command.equals("sync")) {
-                    return;
-                } else if (command.equals("log")) {
-                    String msg = (String) action.get("msg");
-                    LOG.info("Shell msg: " + msg);
-                } else if (command.equals("emit")) {
-                    String stream = (String) action.get("stream");
-                    if (stream == null) stream = Utils.DEFAULT_STREAM_ID;
-                    Long task = (Long) action.get("task");
-                    List<Object> tuple = (List) action.get("tuple");
-                    Object messageId = (Object) action.get("id");
-                    if (task == null) {
-                        List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
-                        Object need_task_ids = action.get("need_task_ids");
-                        if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) {
-                            _process.writeMessage(outtasks);
-                        }
-                    } else {
-                        _collector.emitDirect((int)task.longValue(), stream, tuple, messageId);
-                    }
-                }
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public void activate() {
-    }
-
-    @Override
-    public void deactivate() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
deleted file mode 100644
index 854aa8f..0000000
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ /dev/null
@@ -1,223 +0,0 @@
-package backtype.storm.task;
-
-import backtype.storm.generated.ShellComponent;
-import backtype.storm.tuple.MessageId;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.utils.Utils;
-import backtype.storm.utils.ShellProcess;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import static java.util.concurrent.TimeUnit.SECONDS;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.json.simple.JSONObject;
-
-/**
- * A bolt that shells out to another process to process tuples. ShellBolt
- * communicates with that process over stdio using a special protocol. An ~100
- * line library is required to implement that protocol, and adapter libraries
- * currently exist for Ruby and Python.
- *
- * <p>To run a ShellBolt on a cluster, the scripts that are shelled out to must be
- * in the resources directory within the jar submitted to the master.
- * During development/testing on a local machine, that resources directory just
- * needs to be on the classpath.</p>
- *
- * <p>When creating topologies using the Java API, subclass this bolt and implement
- * the IRichBolt interface to create components for the topology that use other languages. For example:
- * </p>
- *
- * <pre>
- * public class MyBolt extends ShellBolt implements IRichBolt {
- *      public MyBolt() {
- *          super("python", "mybolt.py");
- *      }
- *
- *      public void declareOutputFields(OutputFieldsDeclarer declarer) {
- *          declarer.declare(new Fields("field1", "field2"));
- *      }
- * }
- * </pre>
- */
-public class ShellBolt implements IBolt {
-    public static Logger LOG = LoggerFactory.getLogger(ShellBolt.class);
-    Process _subprocess;
-    OutputCollector _collector;
-    Map<String, Tuple> _inputs = new ConcurrentHashMap<String, Tuple>();
-
-    private String[] _command;
-    private ShellProcess _process;
-    private volatile boolean _running = true;
-    private volatile Throwable _exception;
-    private LinkedBlockingQueue _pendingWrites = new LinkedBlockingQueue();
-    private Random _rand;
-    
-    private Thread _readerThread;
-    private Thread _writerThread;
-
-    public ShellBolt(ShellComponent component) {
-        this(component.get_execution_command(), component.get_script());
-    }
-
-    public ShellBolt(String... command) {
-        _command = command;
-    }
-
-    public void prepare(Map stormConf, TopologyContext context,
-                        final OutputCollector collector) {
-        _rand = new Random();
-        _process = new ShellProcess(_command);
-        _collector = collector;
-
-        try {
-            //subprocesses must send their pid first thing
-            Number subpid = _process.launch(stormConf, context);
-            LOG.info("Launched subprocess with pid " + subpid);
-        } catch (IOException e) {
-            throw new RuntimeException("Error when launching multilang subprocess\n" + _process.getErrorsString(), e);
-        }
-
-        // reader
-        _readerThread = new Thread(new Runnable() {
-            public void run() {
-                while (_running) {
-                    try {
-                        JSONObject action = _process.readMessage();
-                        if (action == null) {
-                            // ignore sync
-                        }
-
-                        String command = (String) action.get("command");
-                        if(command.equals("ack")) {
-                            handleAck(action);
-                        } else if (command.equals("fail")) {
-                            handleFail(action);
-                        } else if (command.equals("error")) {
-                            handleError(action);
-                        } else if (command.equals("log")) {
-                            String msg = (String) action.get("msg");
-                            LOG.info("Shell msg: " + msg);
-                        } else if (command.equals("emit")) {
-                            handleEmit(action);
-                        }
-                    } catch (InterruptedException e) {
-                    } catch (Throwable t) {
-                        die(t);
-                    }
-                }
-            }
-        });
-        
-        _readerThread.start();
-
-        _writerThread = new Thread(new Runnable() {
-            public void run() {
-                while (_running) {
-                    try {
-                        Object write = _pendingWrites.poll(1, SECONDS);
-                        if (write != null) {
-                            _process.writeMessage(write);
-                        }
-                        // drain the error stream to avoid dead lock because of full error stream buffer
-                        _process.drainErrorStream();
-                    } catch (InterruptedException e) {
-                    } catch (Throwable t) {
-                        die(t);
-                    }
-                }
-            }
-        });
-        
-        _writerThread.start();
-    }
-
-    public void execute(Tuple input) {
-        if (_exception != null) {
-            throw new RuntimeException(_exception);
-        }
-
-        //just need an id
-        String genId = Long.toString(_rand.nextLong());
-        _inputs.put(genId, input);
-        try {
-            JSONObject obj = new JSONObject();
-            obj.put("id", genId);
-            obj.put("comp", input.getSourceComponent());
-            obj.put("stream", input.getSourceStreamId());
-            obj.put("task", input.getSourceTask());
-            obj.put("tuple", input.getValues());
-            _pendingWrites.put(obj);
-        } catch(InterruptedException e) {
-            throw new RuntimeException("Error during multilang processing", e);
-        }
-    }
-
-    public void cleanup() {
-        _running = false;
-        _process.destroy();
-        _inputs.clear();
-    }
-
-    private void handleAck(Map action) {
-        String id = (String) action.get("id");
-        Tuple acked = _inputs.remove(id);
-        if(acked==null) {
-            throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id);
-        }
-        _collector.ack(acked);
-    }
-
-    private void handleFail(Map action) {
-        String id = (String) action.get("id");
-        Tuple failed = _inputs.remove(id);
-        if(failed==null) {
-            throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id);
-        }
-        _collector.fail(failed);
-    }
-
-    private void handleError(Map action) {
-        String msg = (String) action.get("msg");
-        _collector.reportError(new Exception("Shell Process Exception: " + msg));
-    }
-
-    private void handleEmit(Map action) throws InterruptedException {
-        String stream = (String) action.get("stream");
-        if(stream==null) stream = Utils.DEFAULT_STREAM_ID;
-        Long task = (Long) action.get("task");
-        List<Object> tuple = (List) action.get("tuple");
-        List<Tuple> anchors = new ArrayList<Tuple>();
-        Object anchorObj = action.get("anchors");
-        if(anchorObj!=null) {
-            if(anchorObj instanceof String) {
-                anchorObj = Arrays.asList(anchorObj);
-            }
-            for(Object o: (List) anchorObj) {
-                Tuple t = _inputs.get((String) o);
-                if (t == null) {
-                    throw new RuntimeException("Anchored onto " + o + " after ack/fail");
-                }
-                anchors.add(t);
-            }
-        }
-        if(task==null) {
-            List<Integer> outtasks = _collector.emit(stream, anchors, tuple);
-            Object need_task_ids = action.get("need_task_ids");
-            if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) {
-                _pendingWrites.put(outtasks);
-            }
-        } else {
-            _collector.emitDirect((int)task.longValue(), stream, anchors, tuple);
-        }
-    }
-
-    private void die(Throwable exception) {
-        _exception = exception;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1a19f3dd/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
deleted file mode 100644
index 49c428a..0000000
--- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
+++ /dev/null
@@ -1,132 +0,0 @@
-package backtype.storm.utils;
-
-import backtype.storm.task.TopologyContext;
-import java.io.BufferedReader;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.util.Map;
-import java.util.List;
-
-import org.apache.commons.io.IOUtils;
-import org.json.simple.JSONObject;
-import org.json.simple.JSONValue;
-import org.apache.log4j.Logger;
-
-public class ShellProcess {
-    public static Logger LOG = Logger.getLogger(ShellProcess.class);
-    private DataOutputStream processIn;
-    private BufferedReader processOut;
-    private InputStream processErrorStream;
-    private Process _subprocess;
-    private String[] command;
-
-    public ShellProcess(String[] command) {
-        this.command = command;
-    }
-
-    public Number launch(Map conf, TopologyContext context) throws IOException {
-        ProcessBuilder builder = new ProcessBuilder(command);
-        builder.directory(new File(context.getCodeDir()));
-        _subprocess = builder.start();
-
-        processIn = new DataOutputStream(_subprocess.getOutputStream());
-        processOut = new BufferedReader(new InputStreamReader(_subprocess.getInputStream()));
-        processErrorStream = _subprocess.getErrorStream();
-
-        JSONObject setupInfo = new JSONObject();
-        setupInfo.put("pidDir", context.getPIDDir());
-        setupInfo.put("conf", conf);
-        setupInfo.put("context", context);
-        writeMessage(setupInfo);
-
-        return (Number)readMessage().get("pid");
-    }
-
-    public void destroy() {
-        _subprocess.destroy();
-    }
-
-    public void writeMessage(Object msg) throws IOException {
-        writeString(JSONValue.toJSONString(msg));
-    }
-
-    private void writeString(String str) throws IOException {
-        byte[] strBytes = str.getBytes("UTF-8");
-        processIn.write(strBytes, 0, strBytes.length);
-        processIn.writeBytes("\nend\n");
-        processIn.flush();
-    }
-
-    public JSONObject readMessage() throws IOException {
-        String string = readString();
-        JSONObject msg = (JSONObject)JSONValue.parse(string);
-        if (msg != null) {
-            return msg;
-        } else {
-            throw new IOException("unable to parse: " + string);
-        }
-    }
-
-    public String getErrorsString() {
-        if(processErrorStream!=null) {
-            try {
-                return IOUtils.toString(processErrorStream);
-            } catch(IOException e) {
-                return "(Unable to capture error stream)";
-            }
-        } else {
-            return "";
-        }
-    }
-
-    public void drainErrorStream()
-    {
-        try {
-            while (processErrorStream.available() > 0)
-            {
-                int bufferSize = processErrorStream.available();
-                byte[] errorReadingBuffer =  new byte[bufferSize];
-
-                processErrorStream.read(errorReadingBuffer, 0, bufferSize);
-
-                LOG.info("Got error from shell process: " + new String(errorReadingBuffer));
-            }
-        } catch(Exception e) {
-        }
-    }
-
-    private String readString() throws IOException {
-        StringBuilder line = new StringBuilder();
-
-        //synchronized (processOut) {
-            while (true) {
-                String subline = processOut.readLine();
-                if(subline==null) {
-                    StringBuilder errorMessage = new StringBuilder();
-                    errorMessage.append("Pipe to subprocess seems to be broken!");
-                    if (line.length() == 0) {
-                        errorMessage.append(" No output read.\n");
-                    }
-                    else {
-                        errorMessage.append(" Currently read output: " + line.toString() + "\n");
-                    }
-                    errorMessage.append("Shell Process Exception:\n");
-                    errorMessage.append(getErrorsString() + "\n");
-                    throw new RuntimeException(errorMessage.toString());
-                }
-                if(subline.equals("end")) {
-                    break;
-                }
-                if(line.length()!=0) {
-                    line.append("\n");
-                }
-                line.append(subline);
-            }
-            //}
-
-        return line.toString();
-    }
-}


[07/24] git commit: Merge branch 'master' of https://github.com/nathanmarz/storm

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/nathanmarz/storm


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

Branch: refs/heads/master
Commit: 18c0ce778bb252e1f8b266d3d3b0a568d283257f
Parents: 06aad3f 1e3d266
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Wed Oct 9 11:38:49 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Wed Oct 9 11:38:49 2013 +0200

----------------------------------------------------------------------
 conf/defaults.yaml                                  |  7 +++++++
 storm-core/src/clj/backtype/storm/testing.clj       | 16 ++++++++++++----
 storm-core/src/jvm/backtype/storm/Config.java       | 12 ++++++++++++
 .../jvm/backtype/storm/messaging/netty/Client.java  |  7 ++++++-
 .../jvm/backtype/storm/messaging/netty/Server.java  |  8 +++++++-
 5 files changed, 44 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/18c0ce77/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/18c0ce77/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------


[03/24] git commit: Added comments to the new multilang objects.

Posted by bo...@apache.org.
Added comments to the new multilang objects.


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

Branch: refs/heads/master
Commit: 3701bfcbe3ed90e1ddf73605a0e5cd0bf4f0dbd4
Parents: a7918f8
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Mon Oct 7 11:47:48 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Mon Oct 7 11:47:48 2013 +0200

----------------------------------------------------------------------
 project.clj                                             |  2 +-
 .../src/jvm/backtype/storm/multilang/Emission.java      | 12 +++++++++++-
 .../src/jvm/backtype/storm/multilang/ISerializer.java   |  5 +++++
 .../src/jvm/backtype/storm/multilang/Immission.java     | 12 +++++++++++-
 .../jvm/backtype/storm/multilang/JsonSerializer.java    |  3 +++
 .../jvm/backtype/storm/multilang/NoOutputException.java |  4 ++++
 .../src/jvm/backtype/storm/multilang/SpoutMsg.java      | 10 ++++++++++
 7 files changed, 45 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3701bfcb/project.clj
----------------------------------------------------------------------
diff --git a/project.clj b/project.clj
index f913924..6e59efc 100644
--- a/project.clj
+++ b/project.clj
@@ -10,7 +10,7 @@
                  :archive "https://groups.google.com/group/storm-user"
                  :post "storm-user@googlegroups.com"}
   :dependencies [~@DEPENDENCIES]
-  :plugins [[~'lein-sub "0.2.1"],[no-man-is-an-island/lein-eclipse "2.0.0"]]
+  :plugins [[~'lein-sub "0.2.1"]]
   :min-lein-version "2.0.0"
   :sub [~@MODULES]
   ))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3701bfcb/storm-core/src/jvm/backtype/storm/multilang/Emission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Emission.java b/storm-core/src/jvm/backtype/storm/multilang/Emission.java
index 1054458..d477d8b 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/Emission.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/Emission.java
@@ -3,6 +3,16 @@ package backtype.storm.multilang;
 import java.util.ArrayList;
 import java.util.List;
 
+/**
+ * Emission is an object that represents the data sent to a shell component
+ * from a process that implements a multi-language protocol. It is the union
+ * of all data types that a component can send to Storm.
+ *
+ * <p> Emissions are objects received from the ISerializer interface, after
+ * the serializer has deserialized the data from the underlying wire protocol.
+ * The Emission class allows for a decoupling between the serialized
+ * representation of the data and the data itself.</p>
+ */
 public class Emission {
 	private String command;
 	private String id;
@@ -11,7 +21,7 @@ public class Emission {
 	private long task;
 	private String msg;
 	private List<Object> tuple;
-	
+
 	public String getCommand() {
 		return command;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3701bfcb/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
index 61a28a9..691b432 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
@@ -9,6 +9,11 @@ import java.util.Map;
 
 import backtype.storm.task.TopologyContext;
 
+/**
+ * The ISerializer interface describes the methods that an object should implement
+ * to provide serialization and de-serialization capabilities to non-JVM
+ * language components.
+ */
 public interface ISerializer extends Serializable {
 	void initialize (OutputStream processIn, InputStream processOut);
 	Number connect (Map conf, TopologyContext context) throws IOException, NoOutputException;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3701bfcb/storm-core/src/jvm/backtype/storm/multilang/Immission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Immission.java b/storm-core/src/jvm/backtype/storm/multilang/Immission.java
index c20df53..968f728 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/Immission.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/Immission.java
@@ -2,13 +2,23 @@ package backtype.storm.multilang;
 
 import java.util.List;
 
+/**
+ * Immission is an object that represents the data sent from a shell component
+ * to a process that implements a multi-language protocol. It is the union
+ * of all data types that a component can receive from Storm.
+ *
+ * <p> Immissions are objects sent to the ISerializer interface, for
+ * serialization according to the wire protocol implemented by the serializer.
+ * The Immission class allows for a decoupling between the serialized
+ * representation of the data and the data itself.</p>
+ */
 public class Immission {
 	private String id;
 	private String comp;
 	private String stream;
 	private long task;
 	private List<Object> tuple;
-	
+
 	public String getId() {
 		return id;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3701bfcb/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
index 88eead2..ba574e5 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -18,6 +18,9 @@ import backtype.storm.task.TopologyContext;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.utils.Utils;
 
+/**
+ * JsonSerializer implements the JSON multilang protocol.
+ */
 public class JsonSerializer implements ISerializer {
 	private DataOutputStream processIn;
     private BufferedReader processOut;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3701bfcb/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
index 595a0ae..8ffb7dd 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
@@ -1,5 +1,9 @@
 package backtype.storm.multilang;
 
+/**
+ * A NoOutputException states that no data has been received from the connected
+ * non-JVM process.
+ */
 public class NoOutputException extends Exception {
 	public NoOutputException() { super(); }
 	public NoOutputException(String message) { super(message); }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3701bfcb/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
index 00967cb..c8063eb 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
@@ -1,5 +1,15 @@
 package backtype.storm.multilang;
 
+/**
+ * SpoutMsg is an object that represents the data sent from a shell spout
+ * to a process that implements a multi-language spout. The SpoutMsg is used
+ * to send a "next", "ack" or "fail" message to a spout.
+ *
+ * <p> Spout messages are objects sent to the ISerializer interface, for
+ * serialization according to the wire protocol implemented by the serializer.
+ * The SpoutMsg class allows for a decoupling between the serialized
+ * representation of the data and the data itself.</p>
+ */
 public class SpoutMsg {
 	private String command;
 	private String id;


[11/24] git commit: Allow for null boltMsg when polling

Posted by bo...@apache.org.
Allow for null boltMsg when polling


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

Branch: refs/heads/master
Commit: 5699373be3842310171aa9b51059d94c03dca82a
Parents: 32b3718
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Wed Oct 9 17:00:10 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Wed Oct 9 17:00:10 2013 +0200

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/task/ShellBolt.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/5699373b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index fb07986..1649fbe 100644
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -116,8 +116,8 @@ public class ShellBolt implements IBolt {
                             _process.writeBoltMsg((BoltMsg)write);
                         } else if (write instanceof List<?>) {
                             _process.writeTaskIds((List<Integer>)write);
-                        } else {
-                            throw new RuntimeException("Cannot write object to bolt:\n" + write.toString());
+                        } else if (write != null) {
+                            throw new RuntimeException("Unknown class type to write: " + write.getClass().getName());
                         }
                     } catch (InterruptedException e) {
                     } catch (Throwable t) {


[08/24] git commit: Added newline to NoOutputException

Posted by bo...@apache.org.
Added newline to NoOutputException


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

Branch: refs/heads/master
Commit: 52948f56511a57cd273431b794819ddc9a343bee
Parents: 18c0ce7
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Wed Oct 9 11:51:20 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Wed Oct 9 11:51:20 2013 +0200

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/52948f56/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
index fd5ef72..768f65f 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
@@ -20,4 +20,4 @@ public class NoOutputException extends Exception {
     public NoOutputException(Throwable cause) {
         super(cause);
     }
-}
\ No newline at end of file
+}


[14/24] git commit: Merge branch 'master' of https://github.com/nathanmarz/storm

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/nathanmarz/storm


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

Branch: refs/heads/master
Commit: 84360dd32c77b050e28bf4c8f3483ce393027c5e
Parents: dfbd045 e8f7e31
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Thu Nov 7 14:46:37 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Thu Nov 7 14:46:37 2013 +0200

----------------------------------------------------------------------
 README.markdown                                 | 32 ++++++++++++++++--
 .../src/clj/backtype/storm/daemon/executor.clj  |  7 ++--
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 20 ++++++-----
 .../test/clj/backtype/storm/tick_tuple_test.clj | 35 ++++++++++++++++++++
 .../backtype/storm/messaging/netty/Client.java  |  5 ++-
 5 files changed, 84 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[19/24] git commit: Merge branch 'master' of github.com:jsgilmore/incubator-storm

Posted by bo...@apache.org.
Merge branch 'master' of github.com:jsgilmore/incubator-storm


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

Branch: refs/heads/master
Commit: f5f41a0b54a852da95a2fa659b1199d727571313
Parents: 46c02be 6a08855
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Wed Apr 23 09:27:39 2014 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Wed Apr 23 09:27:39 2014 +0200

----------------------------------------------------------------------
 CHANGELOG.md                                    |  20 +
 NOTICE                                          |   9 +-
 README.markdown                                 |   5 +-
 bin/storm                                       |  19 +-
 examples/storm-starter/README.markdown          | 108 ++++++
 .../multilang/resources/splitsentence.py        |  24 ++
 .../multilang/resources/splitsentence.rb        |  26 ++
 .../storm-starter/multilang/resources/storm.py  | 221 +++++++++++
 .../storm-starter/multilang/resources/storm.rb  | 200 ++++++++++
 examples/storm-starter/pom.xml                  | 161 ++++++++
 .../src/clj/storm/starter/clj/word_count.clj    |  95 +++++
 .../jvm/storm/starter/BasicDRPCTopology.java    |  78 ++++
 .../jvm/storm/starter/ExclamationTopology.java  |  87 +++++
 .../src/jvm/storm/starter/ManualDRPC.java       |  68 ++++
 .../jvm/storm/starter/PrintSampleStream.java    |  54 +++
 .../src/jvm/storm/starter/ReachTopology.java    | 196 ++++++++++
 .../src/jvm/storm/starter/RollingTopWords.java  |  78 ++++
 .../jvm/storm/starter/SingleJoinExample.java    |  64 ++++
 .../storm/starter/TransactionalGlobalCount.java | 173 +++++++++
 .../jvm/storm/starter/TransactionalWords.java   | 246 +++++++++++++
 .../jvm/storm/starter/WordCountTopology.java    | 107 ++++++
 .../storm/starter/bolt/AbstractRankerBolt.java  | 110 ++++++
 .../starter/bolt/IntermediateRankingsBolt.java  |  58 +++
 .../src/jvm/storm/starter/bolt/PrinterBolt.java |  37 ++
 .../storm/starter/bolt/RollingCountBolt.java    | 142 +++++++
 .../jvm/storm/starter/bolt/SingleJoinBolt.java  | 114 ++++++
 .../storm/starter/bolt/TotalRankingsBolt.java   |  59 +++
 .../starter/spout/RandomSentenceSpout.java      |  64 ++++
 .../storm/starter/spout/TwitterSampleSpout.java | 122 ++++++
 .../tools/NthLastModifiedTimeTracker.java       |  70 ++++
 .../src/jvm/storm/starter/tools/Rankable.java   |  32 ++
 .../starter/tools/RankableObjectWithFields.java | 148 ++++++++
 .../src/jvm/storm/starter/tools/Rankings.java   | 156 ++++++++
 .../starter/tools/SlidingWindowCounter.java     | 119 ++++++
 .../storm/starter/tools/SlotBasedCounter.java   | 118 ++++++
 .../jvm/storm/starter/trident/TridentReach.java | 156 ++++++++
 .../storm/starter/trident/TridentWordCount.java |  85 +++++
 .../src/jvm/storm/starter/util/StormRunner.java |  39 ++
 .../jvm/storm/starter/util/TupleHelpers.java    |  33 ++
 .../bolt/IntermediateRankingsBoltTest.java      | 146 ++++++++
 .../starter/bolt/RollingCountBoltTest.java      | 113 ++++++
 .../starter/bolt/TotalRankingsBoltTest.java     | 147 ++++++++
 .../storm/starter/tools/MockTupleHelpers.java   |  40 ++
 .../tools/NthLastModifiedTimeTrackerTest.java   | 125 +++++++
 .../tools/RankableObjectWithFieldsTest.java     | 252 +++++++++++++
 .../jvm/storm/starter/tools/RankingsTest.java   | 368 +++++++++++++++++++
 .../starter/tools/SlidingWindowCounterTest.java | 106 ++++++
 .../starter/tools/SlotBasedCounterTest.java     | 181 +++++++++
 external/storm-kafka/CHANGELOG.md               |  13 +
 external/storm-kafka/README.md                  |  25 ++
 external/storm-kafka/pom.xml                    | 138 +++++++
 .../storm-kafka/src/jvm/storm/kafka/Broker.java |  80 ++++
 .../src/jvm/storm/kafka/BrokerHosts.java        |  25 ++
 .../jvm/storm/kafka/DynamicBrokersReader.java   | 145 ++++++++
 .../kafka/DynamicPartitionConnections.java      |  94 +++++
 .../jvm/storm/kafka/FailedFetchException.java   |  29 ++
 .../src/jvm/storm/kafka/KafkaConfig.java        |  50 +++
 .../src/jvm/storm/kafka/KafkaError.java         |  43 +++
 .../src/jvm/storm/kafka/KafkaSpout.java         | 190 ++++++++++
 .../src/jvm/storm/kafka/KafkaUtils.java         | 235 ++++++++++++
 .../src/jvm/storm/kafka/KeyValueScheme.java     |  28 ++
 .../kafka/KeyValueSchemeAsMultiScheme.java      |  36 ++
 .../src/jvm/storm/kafka/Partition.java          |  64 ++++
 .../jvm/storm/kafka/PartitionCoordinator.java   |  26 ++
 .../src/jvm/storm/kafka/PartitionManager.java   | 241 ++++++++++++
 .../src/jvm/storm/kafka/SpoutConfig.java        |  36 ++
 .../src/jvm/storm/kafka/StaticCoordinator.java  |  48 +++
 .../src/jvm/storm/kafka/StaticHosts.java        |  38 ++
 .../storm/kafka/StaticPartitionConnections.java |  52 +++
 .../jvm/storm/kafka/StringKeyValueScheme.java   |  37 ++
 .../src/jvm/storm/kafka/StringScheme.java       |  46 +++
 .../src/jvm/storm/kafka/ZkCoordinator.java      | 112 ++++++
 .../src/jvm/storm/kafka/ZkHosts.java            |  36 ++
 .../src/jvm/storm/kafka/ZkState.java            | 116 ++++++
 .../src/jvm/storm/kafka/bolt/KafkaBolt.java     |  89 +++++
 .../jvm/storm/kafka/trident/Coordinator.java    |  50 +++
 .../storm/kafka/trident/DefaultCoordinator.java |  31 ++
 .../trident/GlobalPartitionInformation.java     |  99 +++++
 .../storm/kafka/trident/IBatchCoordinator.java  |  26 ++
 .../jvm/storm/kafka/trident/IBrokerReader.java  |  25 ++
 .../src/jvm/storm/kafka/trident/MaxMetric.java  |  40 ++
 .../kafka/trident/OpaqueTridentKafkaSpout.java  |  59 +++
 .../storm/kafka/trident/StaticBrokerReader.java |  36 ++
 .../trident/TransactionalTridentKafkaSpout.java |  58 +++
 .../storm/kafka/trident/TridentKafkaConfig.java |  37 ++
 .../kafka/trident/TridentKafkaEmitter.java      | 269 ++++++++++++++
 .../jvm/storm/kafka/trident/ZkBrokerReader.java |  62 ++++
 .../storm/kafka/DynamicBrokersReaderTest.java   | 155 ++++++++
 .../src/test/storm/kafka/KafkaErrorTest.java    |  39 ++
 .../src/test/storm/kafka/KafkaTestBroker.java   |  58 +++
 .../src/test/storm/kafka/KafkaUtilsTest.java    | 221 +++++++++++
 .../storm/kafka/StringKeyValueSchemeTest.java   |  38 ++
 .../src/test/storm/kafka/TestUtils.java         |  20 +
 .../src/test/storm/kafka/ZkCoordinatorTest.java | 130 +++++++
 .../test/storm/kafka/bolt/KafkaBoltTest.java    | 171 +++++++++
 pom.xml                                         |  63 +++-
 .../maven-shade-clojure-transformer/pom.xml     |   2 +-
 storm-core/pom.xml                              |  18 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  46 ++-
 .../src/clj/backtype/storm/daemon/worker.clj    |   2 +-
 storm-core/src/clj/backtype/storm/timer.clj     |   2 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  28 +-
 storm-core/src/clj/backtype/storm/util.clj      |   9 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj |   4 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   5 +-
 .../jvm/backtype/storm/ConfigValidation.java    |  17 +
 .../src/jvm/backtype/storm/StormSubmitter.java  |  24 +-
 .../backtype/storm/messaging/netty/Client.java  | 195 ++++++----
 .../backtype/storm/messaging/netty/Context.java |  30 +-
 .../storm/messaging/netty/MessageBatch.java     |  55 +--
 .../backtype/storm/messaging/netty/Server.java  |   2 +-
 .../messaging/netty/StormClientHandler.java     |  80 ++--
 .../src/jvm/backtype/storm/utils/Time.java      |  18 +-
 .../src/jvm/backtype/storm/utils/Utils.java     |  66 ++--
 .../jvm/storm/trident/state/map/OpaqueMap.java  |   6 +-
 .../trident/state/map/RemovableMapState.java    |   8 +
 .../storm/trident/testing/MemoryMapState.java   |  27 +-
 .../test/clj/backtype/storm/config_test.clj     |  20 +
 .../test/clj/backtype/storm/supervisor_test.clj |  64 +++-
 .../test/clj/backtype/storm/utils_test.clj      |  11 +
 .../test/clj/storm/trident/state_test.clj       |  33 +-
 storm-dist/binary/NOTICE                        |   9 +-
 storm-dist/binary/pom.xml                       |   2 +-
 storm-dist/binary/src/main/assembly/binary.xml  |  31 ++
 storm-dist/source/pom.xml                       |   2 +-
 125 files changed, 9462 insertions(+), 292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/f5f41a0b/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------


[02/24] git commit: Moved ShellBolt, ShellProcess and ShellSpout back to their old packages. Updated dependencies.

Posted by bo...@apache.org.
Moved ShellBolt, ShellProcess and ShellSpout back to their old packages. Updated dependencies.


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

Branch: refs/heads/master
Commit: a7918f8ef91b82b5b8fd91c91b36378a6c99caad
Parents: 1a19f3d
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Mon Oct 7 09:33:53 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Mon Oct 7 09:33:53 2013 +0200

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/bootstrap.clj |   7 +-
 .../backtype/storm/clojure/RichShellBolt.java   |   2 +-
 .../backtype/storm/clojure/RichShellSpout.java  |   2 +-
 .../jvm/backtype/storm/multilang/ShellBolt.java | 207 ------------------
 .../backtype/storm/multilang/ShellProcess.java  | 101 ---------
 .../backtype/storm/multilang/ShellSpout.java    | 115 ----------
 .../jvm/backtype/storm/spout/ShellSpout.java    | 117 +++++++++++
 .../src/jvm/backtype/storm/task/ShellBolt.java  | 208 +++++++++++++++++++
 .../jvm/backtype/storm/utils/ShellProcess.java  | 106 ++++++++++
 9 files changed, 436 insertions(+), 429 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/clj/backtype/storm/bootstrap.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index 1075c66..9edbe8e 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -12,13 +12,12 @@
                      RegisteredGlobalState ThriftTopologyUtils DisruptorQueue
                      MutableObject MutableLong]))
      (import (quote [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer]))
-     (import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector]))
-     (import (quote [backtype.storm.multilang ShellSpout ShellBolt]))
+     (import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector ShellSpout]))
      (import (quote [backtype.storm.tuple Tuple TupleImpl Fields MessageId]))
      (import (quote [backtype.storm.task IBolt IOutputCollector
-                     OutputCollector TopologyContext
+                     OutputCollector TopologyContext ShellBolt
                      GeneralTopologyContext WorkerTopologyContext]))
-     (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs 
+     (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs
                      IBatchBolt BatchBoltExecutor]))
      (import (quote [backtype.storm.drpc KeyedFairBolt]))
      (import (quote [backtype.storm.daemon Shutdownable]))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java b/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
index 6ff669e..6be104e 100644
--- a/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
@@ -1,7 +1,7 @@
 package backtype.storm.clojure;
 
 import backtype.storm.generated.StreamInfo;
-import backtype.storm.multilang.ShellBolt;
+import backtype.storm.task.ShellBolt;
 import backtype.storm.topology.IRichBolt;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java b/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
index 57410e1..cb5947f 100644
--- a/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
@@ -1,7 +1,7 @@
 package backtype.storm.clojure;
 
 import backtype.storm.generated.StreamInfo;
-import backtype.storm.multilang.ShellSpout;
+import backtype.storm.spout.ShellSpout;
 import backtype.storm.topology.IRichSpout;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java b/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java
deleted file mode 100644
index acd60c9..0000000
--- a/storm-core/src/jvm/backtype/storm/multilang/ShellBolt.java
+++ /dev/null
@@ -1,207 +0,0 @@
-package backtype.storm.multilang;
-
-import backtype.storm.generated.ShellComponent;
-import backtype.storm.task.IBolt;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.multilang.ShellProcess;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import static java.util.concurrent.TimeUnit.SECONDS;
-import java.util.Map;
-import java.util.Random;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A bolt that shells out to another process to process tuples. ShellBolt
- * communicates with that process over stdio using a special protocol. An ~100
- * line library is required to implement that protocol, and adapter libraries
- * currently exist for Ruby and Python.
- *
- * <p>To run a ShellBolt on a cluster, the scripts that are shelled out to must be
- * in the resources directory within the jar submitted to the master.
- * During development/testing on a local machine, that resources directory just
- * needs to be on the classpath.</p>
- *
- * <p>When creating topologies using the Java API, subclass this bolt and implement
- * the IRichBolt interface to create components for the topology that use other languages. For example:
- * </p>
- *
- * <pre>
- * public class MyBolt extends ShellBolt implements IRichBolt {
- *      public MyBolt() {
- *          super("python", "mybolt.py");
- *      }
- *
- *      public void declareOutputFields(OutputFieldsDeclarer declarer) {
- *          declarer.declare(new Fields("field1", "field2"));
- *      }
- * }
- * </pre>
- */
-public class ShellBolt implements IBolt {
-    public static Logger LOG = LoggerFactory.getLogger(ShellBolt.class);
-    Process _subprocess;
-    OutputCollector _collector;
-    Map<String, Tuple> _inputs = new ConcurrentHashMap<String, Tuple>();
-
-    private String[] _command;
-    private ShellProcess _process;
-    private volatile boolean _running = true;
-    private volatile Throwable _exception;
-    private LinkedBlockingQueue _pendingWrites = new LinkedBlockingQueue();
-    private Random _rand;
-
-    private Thread _readerThread;
-    private Thread _writerThread;
-
-    public ShellBolt(ShellComponent component) {
-        this(component.get_execution_command(), component.get_script());
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellBolt(String... command) {
-        _command = command;
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellBolt(ISerializer serializer, String... command) {
-        _command = command;
-        _process = new ShellProcess(serializer, _command);
-    }
-
-    public void prepare(Map stormConf, TopologyContext context,
-                        final OutputCollector collector) {
-        _rand = new Random();
-        _collector = collector;
-
-        //subprocesses must send their pid first thing
-        Number subpid = _process.launch(stormConf, context);
-        LOG.info("Launched subprocess with pid " + subpid);
-
-        // reader
-        _readerThread = new Thread(new Runnable() {
-            public void run() {
-                while (_running) {
-                    try {
-                        Emission emission = _process.readEmission();
-
-                        String command = emission.getCommand();
-                        if(command.equals("ack")) {
-                            handleAck(emission.getId());
-                        } else if (command.equals("fail")) {
-                            handleFail(emission.getId());
-                        } else if (command.equals("error")) {
-                            handleError(emission.getMsg());
-                        } else if (command.equals("log")) {
-                            String msg = emission.getMsg();
-                            LOG.info("Shell msg: " + msg);
-                        } else if (command.equals("emit")) {
-                            handleEmit(emission);
-                        }
-                    } catch (InterruptedException e) {
-                    } catch (Throwable t) {
-                        die(t);
-                    }
-                }
-            }
-        });
-
-        _readerThread.start();
-
-        _writerThread = new Thread(new Runnable() {
-            public void run() {
-                while (_running) {
-                    try {
-                    	// FIXME: This can either be TaskIds or Immissions
-                        Object write = _pendingWrites.poll(1, SECONDS);
-                        if (write instanceof Immission) {
-                            _process.writeImmission((Immission)write);
-                        } else if (write instanceof List<?>) {
-
-                        }
-                    } catch (InterruptedException e) {
-                    } catch (Throwable t) {
-                        die(t);
-                    }
-                }
-            }
-        });
-
-        _writerThread.start();
-    }
-
-    public void execute(Tuple input) {
-        if (_exception != null) {
-            throw new RuntimeException(_exception);
-        }
-
-        //just need an id
-        String genId = Long.toString(_rand.nextLong());
-        _inputs.put(genId, input);
-        try {
-            Immission immission = new Immission();
-            immission.setId(genId);
-            immission.setComp(input.getSourceComponent());
-            immission.setStream(input.getSourceStreamId());
-            immission.setTask(input.getSourceTask());
-            immission.setTuple(input.getValues());
-
-            _pendingWrites.put(immission);
-        } catch(InterruptedException e) {
-            throw new RuntimeException("Error during multilang processing", e);
-        }
-    }
-
-    public void cleanup() {
-        _running = false;
-        _process.destroy();
-        _inputs.clear();
-    }
-
-    private void handleAck(String id) {
-        Tuple acked = _inputs.remove(id);
-        if(acked==null) {
-            throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id);
-        }
-        _collector.ack(acked);
-    }
-
-    private void handleFail(String id) {
-        Tuple failed = _inputs.remove(id);
-        if(failed==null) {
-            throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id);
-        }
-        _collector.fail(failed);
-    }
-
-    private void handleError(String msg) {
-        _collector.reportError(new Exception("Shell Process Exception: " + msg));
-    }
-
-    private void handleEmit(Emission emission) throws InterruptedException {
-    	List<Tuple> anchors = new ArrayList<Tuple>();
-    	for (String anchor : emission.getAnchors()) {
-	    	Tuple t = _inputs.get(anchor);
-	        if (t == null) {
-	            throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
-	        }
-	        anchors.add(t);
-    	}
-
-        if(emission.getTask() == 0) {
-            List<Integer> outtasks = _collector.emit(emission.getStream(), anchors, emission.getTuple());
-            _pendingWrites.put(outtasks);
-        } else {
-            _collector.emitDirect((int)emission.getTask(), emission.getStream(), anchors, emission.getTuple());
-        }
-    }
-
-    private void die(Throwable exception) {
-        _exception = exception;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java b/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java
deleted file mode 100644
index 12689d8..0000000
--- a/storm-core/src/jvm/backtype/storm/multilang/ShellProcess.java
+++ /dev/null
@@ -1,101 +0,0 @@
-package backtype.storm.multilang;
-
-import backtype.storm.task.TopologyContext;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.log4j.Logger;
-
-public class ShellProcess implements Serializable {
-    public static Logger LOG = Logger.getLogger(ShellProcess.class);
-    private Process _subprocess;
-    private InputStream processErrorStream;
-    private String[] command;
-    public ISerializer serializer;
-
-    public ShellProcess(ISerializer serializer, String[] command) {
-        this.command = command;
-        this.serializer = serializer;
-    }
-
-    public Number launch(Map conf, TopologyContext context) {
-        ProcessBuilder builder = new ProcessBuilder(command);
-        builder.directory(new File(context.getCodeDir()));
-
-        Number pid;
-        try {
-	        _subprocess = builder.start();
-	        processErrorStream = _subprocess.getErrorStream();
-	        serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream());
-	        pid = serializer.connect(conf, context);
-        } catch (IOException e) {
-        	throw new RuntimeException("Error when launching multilang subprocess\n" + getErrorsString(), e);
-        } catch (NoOutputException e) {
-            throw new RuntimeException(e + getErrorsString() + "\n");
-    	}
-        return pid;
-    }
-
-    public void destroy() {
-        _subprocess.destroy();
-    }
-
-    public Emission readEmission() throws IOException {
-    	try  {
-    		return serializer.readEmission();
-    	} catch (NoOutputException e) {
-            throw new RuntimeException(e + getErrorsString() + "\n");
-    	}
-    }
-
-    public void writeImmission(Immission msg) throws IOException {
-    	serializer.writeImmission(msg);
-        // drain the error stream to avoid dead lock because of full error stream buffer
-        drainErrorStream();
-    }
-
-    public void writeSpoutMsg(SpoutMsg msg) throws IOException {
-    	serializer.writeSpoutMsg(msg);
-        // drain the error stream to avoid dead lock because of full error stream buffer
-        drainErrorStream();
-    }
-
-    public void writeTaskIds(List<Integer> taskIds) throws IOException {
-    	serializer.writeTaskIds(taskIds);
-        // drain the error stream to avoid dead lock because of full error stream buffer
-        drainErrorStream();
-    }
-
-    public void drainErrorStream()
-    {
-        try {
-            while (processErrorStream.available() > 0)
-            {
-                int bufferSize = processErrorStream.available();
-                byte[] errorReadingBuffer =  new byte[bufferSize];
-
-                processErrorStream.read(errorReadingBuffer, 0, bufferSize);
-
-                LOG.info("Got error from shell process: " + new String(errorReadingBuffer));
-            }
-        } catch(Exception e) {
-        }
-    }
-
-    public String getErrorsString() {
-        if(processErrorStream!=null) {
-            try {
-                return IOUtils.toString(processErrorStream);
-            } catch(IOException e) {
-                return "(Unable to capture error stream)";
-            }
-        } else {
-            return "";
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java b/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java
deleted file mode 100644
index 7895a00..0000000
--- a/storm-core/src/jvm/backtype/storm/multilang/ShellSpout.java
+++ /dev/null
@@ -1,115 +0,0 @@
-package backtype.storm.multilang;
-
-import backtype.storm.generated.ShellComponent;
-import backtype.storm.spout.ISpout;
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.multilang.ShellProcess;
-import backtype.storm.utils.Utils;
-import java.util.Map;
-import java.util.List;
-import java.io.IOException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class ShellSpout implements ISpout {
-    public static Logger LOG = LoggerFactory.getLogger(ShellSpout.class);
-
-    private SpoutOutputCollector _collector;
-    private String[] _command;
-    private ShellProcess _process;
-    private SpoutMsg spoutMsg;
-
-    public ShellSpout(ShellComponent component) {
-        this(component.get_execution_command(), component.get_script());
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellSpout(String... command) {
-        _command = command;
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellSpout(ISerializer serializer, String... command) {
-        _command = command;
-        _process = new ShellProcess(serializer, _command);
-    }
-
-    public void open(Map stormConf, TopologyContext context,
-                     SpoutOutputCollector collector) {
-        _collector = collector;
-
-        Number subpid = _process.launch(stormConf, context);
-        LOG.info("Launched subprocess with pid " + subpid);
-    }
-
-    public void close() {
-        _process.destroy();
-    }
-
-    public void nextTuple() {
-        if (spoutMsg == null) {
-            spoutMsg = new SpoutMsg();
-        }
-        spoutMsg.setCommand("next");
-        spoutMsg.setId("");
-        querySubprocess();
-    }
-
-    public void ack(Object msgId) {
-    	if (spoutMsg == null) {
-            spoutMsg = new SpoutMsg();
-        }
-        spoutMsg.setCommand("ack");
-        spoutMsg.setId(msgId.toString());
-        querySubprocess();
-    }
-
-    public void fail(Object msgId) {
-    	if (spoutMsg == null) {
-            spoutMsg = new SpoutMsg();
-        }
-        spoutMsg.setCommand("fail");
-        spoutMsg.setId(msgId.toString());
-        querySubprocess();
-    }
-
-    private void querySubprocess() {
-        try {
-            _process.writeSpoutMsg(spoutMsg);
-
-            while (true) {
-                Emission emission = _process.readEmission();
-                String command = emission.getCommand();
-                if (command.equals("sync")) {
-                    return;
-                } else if (command.equals("log")) {
-                    String msg = emission.getMsg();
-                    LOG.info("Shell msg: " + msg);
-                } else if (command.equals("emit")) {
-                    String stream = emission.getStream();
-                    Long task = emission.getTask();
-                    List<Object> tuple = emission.getTuple();
-                    Object messageId = emission.getId();
-                    if (task == 0) {
-                        List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
-                        _process.writeTaskIds(outtasks);
-                    } else {
-                        _collector.emitDirect((int)task.longValue(), stream, tuple, messageId);
-                    }
-                }
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public void activate() {
-    }
-
-    @Override
-    public void deactivate() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
new file mode 100644
index 0000000..c9ef682
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -0,0 +1,117 @@
+package backtype.storm.spout;
+
+import backtype.storm.generated.ShellComponent;
+import backtype.storm.multilang.Emission;
+import backtype.storm.multilang.ISerializer;
+import backtype.storm.multilang.JsonSerializer;
+import backtype.storm.multilang.SpoutMsg;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.utils.ShellProcess;
+import backtype.storm.utils.Utils;
+import java.util.Map;
+import java.util.List;
+import java.io.IOException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ShellSpout implements ISpout {
+    public static Logger LOG = LoggerFactory.getLogger(ShellSpout.class);
+
+    private SpoutOutputCollector _collector;
+    private String[] _command;
+    private ShellProcess _process;
+    private SpoutMsg spoutMsg;
+
+    public ShellSpout(ShellComponent component) {
+        this(component.get_execution_command(), component.get_script());
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellSpout(String... command) {
+        _command = command;
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellSpout(ISerializer serializer, String... command) {
+        _command = command;
+        _process = new ShellProcess(serializer, _command);
+    }
+
+    public void open(Map stormConf, TopologyContext context,
+                     SpoutOutputCollector collector) {
+        _collector = collector;
+
+        Number subpid = _process.launch(stormConf, context);
+        LOG.info("Launched subprocess with pid " + subpid);
+    }
+
+    public void close() {
+        _process.destroy();
+    }
+
+    public void nextTuple() {
+        if (spoutMsg == null) {
+            spoutMsg = new SpoutMsg();
+        }
+        spoutMsg.setCommand("next");
+        spoutMsg.setId("");
+        querySubprocess();
+    }
+
+    public void ack(Object msgId) {
+    	if (spoutMsg == null) {
+            spoutMsg = new SpoutMsg();
+        }
+        spoutMsg.setCommand("ack");
+        spoutMsg.setId(msgId.toString());
+        querySubprocess();
+    }
+
+    public void fail(Object msgId) {
+    	if (spoutMsg == null) {
+            spoutMsg = new SpoutMsg();
+        }
+        spoutMsg.setCommand("fail");
+        spoutMsg.setId(msgId.toString());
+        querySubprocess();
+    }
+
+    private void querySubprocess() {
+        try {
+            _process.writeSpoutMsg(spoutMsg);
+
+            while (true) {
+                Emission emission = _process.readEmission();
+                String command = emission.getCommand();
+                if (command.equals("sync")) {
+                    return;
+                } else if (command.equals("log")) {
+                    String msg = emission.getMsg();
+                    LOG.info("Shell msg: " + msg);
+                } else if (command.equals("emit")) {
+                    String stream = emission.getStream();
+                    Long task = emission.getTask();
+                    List<Object> tuple = emission.getTuple();
+                    Object messageId = emission.getId();
+                    if (task == 0) {
+                        List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
+                        _process.writeTaskIds(outtasks);
+                    } else {
+                        _collector.emitDirect((int)task.longValue(), stream, tuple, messageId);
+                    }
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void activate() {
+    }
+
+    @Override
+    public void deactivate() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
new file mode 100644
index 0000000..36585e3
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -0,0 +1,208 @@
+package backtype.storm.task;
+
+import backtype.storm.generated.ShellComponent;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.ShellProcess;
+import backtype.storm.multilang.Emission;
+import backtype.storm.multilang.ISerializer;
+import backtype.storm.multilang.Immission;
+import backtype.storm.multilang.JsonSerializer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import java.util.Map;
+import java.util.Random;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A bolt that shells out to another process to process tuples. ShellBolt
+ * communicates with that process over stdio using a special protocol. An ~100
+ * line library is required to implement that protocol, and adapter libraries
+ * currently exist for Ruby and Python.
+ *
+ * <p>To run a ShellBolt on a cluster, the scripts that are shelled out to must be
+ * in the resources directory within the jar submitted to the master.
+ * During development/testing on a local machine, that resources directory just
+ * needs to be on the classpath.</p>
+ *
+ * <p>When creating topologies using the Java API, subclass this bolt and implement
+ * the IRichBolt interface to create components for the topology that use other languages. For example:
+ * </p>
+ *
+ * <pre>
+ * public class MyBolt extends ShellBolt implements IRichBolt {
+ *      public MyBolt() {
+ *          super("python", "mybolt.py");
+ *      }
+ *
+ *      public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ *          declarer.declare(new Fields("field1", "field2"));
+ *      }
+ * }
+ * </pre>
+ */
+public class ShellBolt implements IBolt {
+    public static Logger LOG = LoggerFactory.getLogger(ShellBolt.class);
+    Process _subprocess;
+    OutputCollector _collector;
+    Map<String, Tuple> _inputs = new ConcurrentHashMap<String, Tuple>();
+
+    private String[] _command;
+    private ShellProcess _process;
+    private volatile boolean _running = true;
+    private volatile Throwable _exception;
+    private LinkedBlockingQueue _pendingWrites = new LinkedBlockingQueue();
+    private Random _rand;
+
+    private Thread _readerThread;
+    private Thread _writerThread;
+
+    public ShellBolt(ShellComponent component) {
+        this(component.get_execution_command(), component.get_script());
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellBolt(String... command) {
+        _command = command;
+        _process = new ShellProcess(new JsonSerializer(), _command);
+    }
+
+    public ShellBolt(ISerializer serializer, String... command) {
+        _command = command;
+        _process = new ShellProcess(serializer, _command);
+    }
+
+    public void prepare(Map stormConf, TopologyContext context,
+                        final OutputCollector collector) {
+        _rand = new Random();
+        _collector = collector;
+
+        //subprocesses must send their pid first thing
+        Number subpid = _process.launch(stormConf, context);
+        LOG.info("Launched subprocess with pid " + subpid);
+
+        // reader
+        _readerThread = new Thread(new Runnable() {
+            public void run() {
+                while (_running) {
+                    try {
+                        Emission emission = _process.readEmission();
+
+                        String command = emission.getCommand();
+                        if(command.equals("ack")) {
+                            handleAck(emission.getId());
+                        } else if (command.equals("fail")) {
+                            handleFail(emission.getId());
+                        } else if (command.equals("error")) {
+                            handleError(emission.getMsg());
+                        } else if (command.equals("log")) {
+                            String msg = emission.getMsg();
+                            LOG.info("Shell msg: " + msg);
+                        } else if (command.equals("emit")) {
+                            handleEmit(emission);
+                        }
+                    } catch (InterruptedException e) {
+                    } catch (Throwable t) {
+                        die(t);
+                    }
+                }
+            }
+        });
+
+        _readerThread.start();
+
+        _writerThread = new Thread(new Runnable() {
+            public void run() {
+                while (_running) {
+                    try {
+                    	// FIXME: This can either be TaskIds or Immissions
+                        Object write = _pendingWrites.poll(1, SECONDS);
+                        if (write instanceof Immission) {
+                            _process.writeImmission((Immission)write);
+                        } else if (write instanceof List<?>) {
+
+                        }
+                    } catch (InterruptedException e) {
+                    } catch (Throwable t) {
+                        die(t);
+                    }
+                }
+            }
+        });
+
+        _writerThread.start();
+    }
+
+    public void execute(Tuple input) {
+        if (_exception != null) {
+            throw new RuntimeException(_exception);
+        }
+
+        //just need an id
+        String genId = Long.toString(_rand.nextLong());
+        _inputs.put(genId, input);
+        try {
+            Immission immission = new Immission();
+            immission.setId(genId);
+            immission.setComp(input.getSourceComponent());
+            immission.setStream(input.getSourceStreamId());
+            immission.setTask(input.getSourceTask());
+            immission.setTuple(input.getValues());
+
+            _pendingWrites.put(immission);
+        } catch(InterruptedException e) {
+            throw new RuntimeException("Error during multilang processing", e);
+        }
+    }
+
+    public void cleanup() {
+        _running = false;
+        _process.destroy();
+        _inputs.clear();
+    }
+
+    private void handleAck(String id) {
+        Tuple acked = _inputs.remove(id);
+        if(acked==null) {
+            throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id);
+        }
+        _collector.ack(acked);
+    }
+
+    private void handleFail(String id) {
+        Tuple failed = _inputs.remove(id);
+        if(failed==null) {
+            throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id);
+        }
+        _collector.fail(failed);
+    }
+
+    private void handleError(String msg) {
+        _collector.reportError(new Exception("Shell Process Exception: " + msg));
+    }
+
+    private void handleEmit(Emission emission) throws InterruptedException {
+    	List<Tuple> anchors = new ArrayList<Tuple>();
+    	for (String anchor : emission.getAnchors()) {
+	    	Tuple t = _inputs.get(anchor);
+	        if (t == null) {
+	            throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
+	        }
+	        anchors.add(t);
+    	}
+
+        if(emission.getTask() == 0) {
+            List<Integer> outtasks = _collector.emit(emission.getStream(), anchors, emission.getTuple());
+            _pendingWrites.put(outtasks);
+        } else {
+            _collector.emitDirect((int)emission.getTask(), emission.getStream(), anchors, emission.getTuple());
+        }
+    }
+
+    private void die(Throwable exception) {
+        _exception = exception;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a7918f8e/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
new file mode 100644
index 0000000..38976bc
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
@@ -0,0 +1,106 @@
+package backtype.storm.utils;
+
+import backtype.storm.multilang.Emission;
+import backtype.storm.multilang.ISerializer;
+import backtype.storm.multilang.Immission;
+import backtype.storm.multilang.NoOutputException;
+import backtype.storm.multilang.SpoutMsg;
+import backtype.storm.task.TopologyContext;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.log4j.Logger;
+
+public class ShellProcess implements Serializable {
+    public static Logger LOG = Logger.getLogger(ShellProcess.class);
+    private Process _subprocess;
+    private InputStream processErrorStream;
+    private String[] command;
+    public ISerializer serializer;
+
+    public ShellProcess(ISerializer serializer, String[] command) {
+        this.command = command;
+        this.serializer = serializer;
+    }
+
+    public Number launch(Map conf, TopologyContext context) {
+        ProcessBuilder builder = new ProcessBuilder(command);
+        builder.directory(new File(context.getCodeDir()));
+
+        Number pid;
+        try {
+	        _subprocess = builder.start();
+	        processErrorStream = _subprocess.getErrorStream();
+	        serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream());
+	        pid = serializer.connect(conf, context);
+        } catch (IOException e) {
+        	throw new RuntimeException("Error when launching multilang subprocess\n" + getErrorsString(), e);
+        } catch (NoOutputException e) {
+            throw new RuntimeException(e + getErrorsString() + "\n");
+    	}
+        return pid;
+    }
+
+    public void destroy() {
+        _subprocess.destroy();
+    }
+
+    public Emission readEmission() throws IOException {
+    	try  {
+    		return serializer.readEmission();
+    	} catch (NoOutputException e) {
+            throw new RuntimeException(e + getErrorsString() + "\n");
+    	}
+    }
+
+    public void writeImmission(Immission msg) throws IOException {
+    	serializer.writeImmission(msg);
+        // drain the error stream to avoid dead lock because of full error stream buffer
+        drainErrorStream();
+    }
+
+    public void writeSpoutMsg(SpoutMsg msg) throws IOException {
+    	serializer.writeSpoutMsg(msg);
+        // drain the error stream to avoid dead lock because of full error stream buffer
+        drainErrorStream();
+    }
+
+    public void writeTaskIds(List<Integer> taskIds) throws IOException {
+    	serializer.writeTaskIds(taskIds);
+        // drain the error stream to avoid dead lock because of full error stream buffer
+        drainErrorStream();
+    }
+
+    public void drainErrorStream()
+    {
+        try {
+            while (processErrorStream.available() > 0)
+            {
+                int bufferSize = processErrorStream.available();
+                byte[] errorReadingBuffer =  new byte[bufferSize];
+
+                processErrorStream.read(errorReadingBuffer, 0, bufferSize);
+
+                LOG.info("Got error from shell process: " + new String(errorReadingBuffer));
+            }
+        } catch(Exception e) {
+        }
+    }
+
+    public String getErrorsString() {
+        if(processErrorStream!=null) {
+            try {
+                return IOUtils.toString(processErrorStream);
+            } catch(IOException e) {
+                return "(Unable to capture error stream)";
+            }
+        } else {
+            return "";
+        }
+    }
+}


[22/24] git commit: Removed sync comment and changed spoutMsg name to _spoutMsg

Posted by bo...@apache.org.
Removed sync comment and changed spoutMsg name to _spoutMsg


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

Branch: refs/heads/master
Commit: bcbd22bf3527fb700b4dda06207b22638db5a16d
Parents: 0c669f4
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Thu May 15 07:04:15 2014 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Thu May 15 07:04:15 2014 +0200

----------------------------------------------------------------------
 .../storm/multilang/JsonSerializer.java         |  2 --
 .../jvm/backtype/storm/spout/ShellSpout.java    | 28 ++++++++++----------
 2 files changed, 14 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/bcbd22bf/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
index 0d79659..dd5773f 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -135,7 +135,6 @@ public class JsonSerializer implements ISerializer {
     private String readString() throws IOException, NoOutputException {
         StringBuilder line = new StringBuilder();
 
-        // synchronized (processOut) {
         while (true) {
             String subline = processOut.readLine();
             if (subline == null) {
@@ -158,7 +157,6 @@ public class JsonSerializer implements ISerializer {
             }
             line.append(subline);
         }
-        // }
         return line.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/bcbd22bf/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index 87bc5a8..d6a18e7 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -35,7 +35,7 @@ public class ShellSpout implements ISpout {
     private SpoutOutputCollector _collector;
     private String[] _command;
     private ShellProcess _process;
-    private SpoutMsg spoutMsg;
+    private SpoutMsg _spoutMsg;
 
     public ShellSpout(ShellComponent component) {
         this(component.get_execution_command(), component.get_script());
@@ -60,35 +60,35 @@ public class ShellSpout implements ISpout {
     }
 
     public void nextTuple() {
-        if (spoutMsg == null) {
-            spoutMsg = new SpoutMsg();
+        if (_spoutMsg == null) {
+            _spoutMsg = new SpoutMsg();
         }
-        spoutMsg.setCommand("next");
-        spoutMsg.setId("");
+        _spoutMsg.setCommand("next");
+        _spoutMsg.setId("");
         querySubprocess();
     }
 
     public void ack(Object msgId) {
-        if (spoutMsg == null) {
-            spoutMsg = new SpoutMsg();
+        if (_spoutMsg == null) {
+            _spoutMsg = new SpoutMsg();
         }
-        spoutMsg.setCommand("ack");
-        spoutMsg.setId(msgId);
+        _spoutMsg.setCommand("ack");
+        _spoutMsg.setId(msgId);
         querySubprocess();
     }
 
     public void fail(Object msgId) {
-        if (spoutMsg == null) {
-            spoutMsg = new SpoutMsg();
+        if (_spoutMsg == null) {
+            _spoutMsg = new SpoutMsg();
         }
-        spoutMsg.setCommand("fail");
-        spoutMsg.setId(msgId);
+        _spoutMsg.setCommand("fail");
+        _spoutMsg.setId(msgId);
         querySubprocess();
     }
 
     private void querySubprocess() {
         try {
-            _process.writeSpoutMsg(spoutMsg);
+            _process.writeSpoutMsg(_spoutMsg);
 
             while (true) {
                 ShellMsg shellMsg = _process.readShellMsg();


[09/24] git commit: Re-added need_task_ids option

Posted by bo...@apache.org.
Re-added need_task_ids option


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

Branch: refs/heads/master
Commit: 63afb94cd0d8aa769898ba80ed2b020a6379b52f
Parents: 52948f5
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Wed Oct 9 15:29:21 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Wed Oct 9 15:29:21 2013 +0200

----------------------------------------------------------------------
 .../src/jvm/backtype/storm/multilang/JsonSerializer.java    | 9 +++++++++
 storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java   | 9 +++++++++
 storm-core/src/jvm/backtype/storm/spout/ShellSpout.java     | 4 +++-
 storm-core/src/jvm/backtype/storm/task/ShellBolt.java       | 4 +++-
 4 files changed, 24 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/63afb94c/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
index ceb8cb3..8afa4bb 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -89,6 +89,15 @@ public class JsonSerializer implements ISerializer {
         Object taskObj = msg.get("task");
         if (taskObj != null) {
             shellMsg.setTask((Long) taskObj);
+        } else {
+            shellMsg.setTask(0);
+        }
+        
+        Object need_task_ids = msg.get("need_task_ids");
+        if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) {
+            shellMsg.setNeedTaskIds(true);
+        } else {
+            shellMsg.setNeedTaskIds(false);
         }
 
         shellMsg.setTuple((List) msg.get("tuple"));

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/63afb94c/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
index d56ba78..4381c07 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
@@ -23,6 +23,7 @@ public class ShellMsg {
     private long task;
     private String msg;
     private List<Object> tuple;
+    private boolean needTaskIds;
 
     public String getCommand() {
         return command;
@@ -93,4 +94,12 @@ public class ShellMsg {
         }
         this.tuple.add(tuple);
     }
+
+    public boolean areTaskIdsNeeded() {
+        return needTaskIds;
+    }
+
+    public void setNeedTaskIds(boolean needTaskIds) {
+        this.needTaskIds = needTaskIds;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/63afb94c/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index 72cebe2..bee56fe 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -87,7 +87,9 @@ public class ShellSpout implements ISpout {
                     Object messageId = shellMsg.getId();
                     if (task == 0) {
                         List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
-                        _process.writeTaskIds(outtasks);
+                        if (shellMsg.areTaskIdsNeeded()) {
+                            _process.writeTaskIds(outtasks);
+                        }
                     } else {
                         _collector.emitDirect((int) task.longValue(), stream,
                                 tuple, messageId);

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/63afb94c/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index 6a6c305..fb07986 100644
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -190,7 +190,9 @@ public class ShellBolt implements IBolt {
 
         if(shellMsg.getTask() == 0) {
             List<Integer> outtasks = _collector.emit(shellMsg.getStream(), anchors, shellMsg.getTuple());
-            _pendingWrites.put(outtasks);
+            if (shellMsg.areTaskIdsNeeded()) {
+                _pendingWrites.put(outtasks);
+            }
         } else {
             _collector.emitDirect((int) shellMsg.getTask(),
                     shellMsg.getStream(), anchors, shellMsg.getTuple());


[23/24] git commit: Merge branch 'master' of https://github.com/jsgilmore/incubator-storm into STORM-138

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/jsgilmore/incubator-storm into STORM-138

Conflicts:
	storm-core/src/jvm/backtype/storm/Config.java

STORM-138: Pluggable serialization for multilang


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

Branch: refs/heads/master
Commit: d3c6a1b1e8d03297cdca95f9244d7c49a590e3fa
Parents: 1311a1d bcbd22b
Author: Robert (Bobby) Evans <bo...@apache.org>
Authored: Mon May 19 21:39:19 2014 +0000
Committer: Robert (Bobby) Evans <bo...@apache.org>
Committed: Mon May 19 21:39:19 2014 +0000

----------------------------------------------------------------------
 conf/defaults.yaml                              |   1 +
 storm-core/src/jvm/backtype/storm/Config.java   |   7 +
 .../jvm/backtype/storm/multilang/BoltMsg.java   |  63 +++++++
 .../backtype/storm/multilang/ISerializer.java   |  65 +++++++
 .../storm/multilang/JsonSerializer.java         | 162 ++++++++++++++++++
 .../storm/multilang/NoOutputException.java      |  23 +++
 .../jvm/backtype/storm/multilang/ShellMsg.java  | 105 ++++++++++++
 .../jvm/backtype/storm/multilang/SpoutMsg.java  |  34 ++++
 .../jvm/backtype/storm/spout/ShellSpout.java    |  85 +++++-----
 .../src/jvm/backtype/storm/task/ShellBolt.java  | 109 ++++++------
 .../jvm/backtype/storm/utils/ShellProcess.java  | 169 +++++++++----------
 11 files changed, 628 insertions(+), 195 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/d3c6a1b1/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/d3c6a1b1/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/Config.java
index 9b91e07,593898f..83ee232
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@@ -500,6 -500,20 +500,13 @@@ public class Config extends HashMap<Str
      public static final Object TOPOLOGY_DEBUG_SCHEMA = Boolean.class;
  
      /**
+      * The serializer for communication between shell components and non-JVM
+      * processes
+      */
+     public static final String TOPOLOGY_MULTILANG_SERIALIZER = "topology.multilang.serializer";
+     public static final Object TOPOLOGY_MULTILANG_SERIALIZER_SCHEMA = String.class;
+ 
+     /**
 -     * Whether or not the master should optimize topologies by running multiple
 -     * tasks in a single thread where appropriate.
 -     */
 -    public static final String TOPOLOGY_OPTIMIZE = "topology.optimize";
 -    public static final Object TOPOLOGY_OPTIMIZE_SCHEMA = Boolean.class;
 -
 -    /**
       * How many processes should be spawned around the cluster to execute this
       * topology. Each process will execute some number of tasks as threads within
       * them. This parameter should be used in conjunction with the parallelism hints


[05/24] git commit: multilang serializer is now handled by stormConf. Renamed Emission and Immission to ShellMsg and BoltMsg respectively. Documented ISerializer interface.

Posted by bo...@apache.org.
multilang serializer is now handled by stormConf. Renamed Emission and Immission to ShellMsg and BoltMsg respectively. Documented ISerializer interface.


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

Branch: refs/heads/master
Commit: ee0678c7f8bed531a8514166c7254a2190f3c4a9
Parents: 3701bfc
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Tue Oct 8 13:52:00 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Tue Oct 8 13:53:32 2013 +0200

----------------------------------------------------------------------
 conf/defaults.yaml                              |   1 +
 storm-core/src/jvm/backtype/storm/Config.java   | 129 +++++++++--------
 .../jvm/backtype/storm/multilang/BoltMsg.java   |  63 +++++++++
 .../jvm/backtype/storm/multilang/Emission.java  |  79 -----------
 .../backtype/storm/multilang/ISerializer.java   |  59 ++++++--
 .../jvm/backtype/storm/multilang/Immission.java |  52 -------
 .../storm/multilang/JsonSerializer.java         | 138 ++++++++++---------
 .../storm/multilang/NoOutputException.java      |  19 ++-
 .../jvm/backtype/storm/multilang/ShellMsg.java  |  96 +++++++++++++
 .../jvm/backtype/storm/multilang/SpoutMsg.java  |  44 +++---
 .../jvm/backtype/storm/spout/ShellSpout.java    |  30 ++--
 .../src/jvm/backtype/storm/task/ShellBolt.java  |  65 ++++-----
 .../jvm/backtype/storm/utils/ShellProcess.java  |  95 ++++++++-----
 13 files changed, 489 insertions(+), 381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index b17e445..92e65d6 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -19,6 +19,7 @@ storm.cluster.mode: "distributed" # can be distributed or local
 storm.local.mode.zmq: false
 storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
 storm.messaging.transport: "backtype.storm.messaging.zmq"
+storm.multilang.serializer: "backtype.storm.multilang.JsonSerializer"
 
 ### nimbus.* configs are for the master
 nimbus.host: "localhost"

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/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 564ad0d..fff1573 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -10,11 +10,11 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Topology configs are specified as a plain old map. This class provides a 
- * convenient way to create a topology config map by providing setter methods for 
- * all the configs that can be set. It also makes it easier to do things like add 
+ * Topology configs are specified as a plain old map. This class provides a
+ * convenient way to create a topology config map by providing setter methods for
+ * all the configs that can be set. It also makes it easier to do things like add
  * serializations.
- * 
+ *
  * <p>This class also provides constants for all the configurations possible on
  * a Storm cluster and Storm topology. Each constant is paired with a schema
  * that defines the validity criterion of the corresponding field. Default
@@ -22,11 +22,18 @@ import java.util.Map;
  *
  * <p>Note that you may put other configurations in any of the configs. Storm
  * will ignore anything it doesn't recognize, but your topologies are free to make
- * use of them by reading them in the prepare method of Bolts or the open method of 
+ * use of them by reading them in the prepare method of Bolts or the open method of
  * Spouts.</p>
  */
 public class Config extends HashMap<String, Object> {
     /**
+     * The serializer for communication between shell components and non-JVM
+     * processes
+     */
+    public static final String STORM_MULTILANG_SERIALIZER = "storm.multilang.serializer";
+    public static final Object STORM_MULTILANG_SERIALIZER_SCHEMA = String.class;
+
+    /**
      * The transporter for communication among Storm tasks
      */
     public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport";
@@ -35,25 +42,25 @@ public class Config extends HashMap<String, Object> {
     /**
      * Netty based messaging: The buffer size for send/recv buffer
      */
-    public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; 
+    public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size";
     public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = Number.class;
 
     /**
      * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible
      */
-    public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; 
+    public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries";
     public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = Number.class;
 
     /**
-     * Netty based messaging: The min # of milliseconds that a peer will wait. 
+     * Netty based messaging: The min # of milliseconds that a peer will wait.
      */
-    public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; 
+    public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms";
     public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = Number.class;
 
     /**
-     * Netty based messaging: The max # of milliseconds that a peer will wait. 
+     * Netty based messaging: The max # of milliseconds that a peer will wait.
      */
-    public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; 
+    public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms";
     public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = Number.class;
 
     /**
@@ -78,7 +85,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A global task scheduler used to assign topologies's tasks to supervisors' wokers.
-     * 
+     *
      * If this is not set, a default system scheduler will be used.
      */
     public static final String STORM_SCHEDULER = "storm.scheduler";
@@ -91,9 +98,9 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class;
 
     /**
-     * The hostname the supervisors/workers should report to nimbus. If unset, Storm will 
+     * The hostname the supervisors/workers should report to nimbus. If unset, Storm will
      * get the hostname to report by calling <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
-     * 
+     *
      * You should set this config when you dont have a DNS which supervisors/workers
      * can utilize to find each other based on hostname got from calls to
      * <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
@@ -108,16 +115,16 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
-     * The serializer class for ListDelegate (tuple payload). 
+     * The serializer class for ListDelegate (tuple payload).
      * The default serializer will be ListDelegateSerializer
      */
     public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer";
     public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class;
 
     /**
-     * Whether or not to use ZeroMQ for messaging in local mode. If this is set 
-     * to false, then Storm will use a pure-Java messaging system. The purpose 
-     * of this flag is to make it easy to run Storm in local mode by eliminating 
+     * Whether or not to use ZeroMQ for messaging in local mode. If this is set
+     * to false, then Storm will use a pure-Java messaging system. The purpose
+     * of this flag is to make it easy to run Storm in local mode by eliminating
      * the need for native dependencies, which can be difficult to install.
      *
      * Defaults to false.
@@ -255,7 +262,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = Number.class;
 
     /**
-     * Whether or not nimbus should reassign tasks if it detects that a task goes down. 
+     * Whether or not nimbus should reassign tasks if it detects that a task goes down.
      * Defaults to true, and it's not recommended to change this value.
      */
     public static final String NIMBUS_REASSIGN = "nimbus.reassign";
@@ -319,19 +326,19 @@ public class Config extends HashMap<String, Object> {
     public static final Object DRPC_PORT_SCHEMA = Number.class;
 
     /**
-     * DRPC thrift server worker threads 
+     * DRPC thrift server worker threads
      */
     public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
     public static final Object DRPC_WORKER_THREADS_SCHEMA = Number.class;
 
     /**
-     * DRPC thrift server queue size 
+     * DRPC thrift server queue size
      */
     public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
     public static final Object DRPC_QUEUE_SIZE_SCHEMA = Number.class;
 
     /**
-     * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back. 
+     * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.
      */
     public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
     public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = Number.class;
@@ -479,8 +486,8 @@ public class Config extends HashMap<String, Object> {
     /**
      * How many instances to create for a spout/bolt. A task runs on a thread with zero or more
      * other tasks for the same spout/bolt. The number of tasks for a spout/bolt is always
-     * the same throughout the lifetime of a topology, but the number of executors (threads) for 
-     * a spout/bolt can change over time. This allows a topology to scale to more or less resources 
+     * the same throughout the lifetime of a topology, but the number of executors (threads) for
+     * a spout/bolt can change over time. This allows a topology to scale to more or less resources
      * without redeploying the topology or violating the constraints of Storm (such as a fields grouping
      * guaranteeing that the same value goes to the same task).
      */
@@ -519,8 +526,8 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A list of classes that customize storm's kryo instance during start-up.
-     * Each listed class name must implement IKryoDecorator. During start-up the 
-     * listed class is instantiated with 0 arguments, then its 'decorate' method 
+     * Each listed class name must implement IKryoDecorator. During start-up the
+     * listed class is instantiated with 0 arguments, then its 'decorate' method
      * is called with storm's kryo instance as the only argument.
      */
     public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators";
@@ -550,7 +557,7 @@ public class Config extends HashMap<String, Object> {
 
     /*
      * A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format).
-     * Each listed class will be routed all the metrics data generated by the storm metrics API. 
+     * Each listed class will be routed all the metrics data generated by the storm metrics API.
      * Each listed class maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable.
      */
     public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register";
@@ -566,24 +573,24 @@ public class Config extends HashMap<String, Object> {
 
 
     /**
-     * The maximum number of tuples that can be pending on a spout task at any given time. 
-     * This config applies to individual tasks, not to spouts or topologies as a whole. 
-     * 
+     * The maximum number of tuples that can be pending on a spout task at any given time.
+     * This config applies to individual tasks, not to spouts or topologies as a whole.
+     *
      * A pending tuple is one that has been emitted from a spout but has not been acked or failed yet.
-     * Note that this config parameter has no effect for unreliable spouts that don't tag 
+     * Note that this config parameter has no effect for unreliable spouts that don't tag
      * their tuples with a message id.
      */
-    public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; 
+    public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending";
     public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = Number.class;
 
     /**
      * A class that implements a strategy for what to do when a spout needs to wait. Waiting is
      * triggered in one of two conditions:
-     * 
+     *
      * 1. nextTuple emits no tuples
      * 2. The spout has hit maxSpoutPending and can't emit any more tuples
      */
-    public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy"; 
+    public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy";
     public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class;
 
     /**
@@ -606,7 +613,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA = Number.class;
 
     /**
-     * The time period that builtin metrics data in bucketed into. 
+     * The time period that builtin metrics data in bucketed into.
      */
     public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs";
     public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = Number.class;
@@ -633,7 +640,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A list of task hooks that are automatically added to every spout and bolt in the topology. An example
-     * of when you'd do this is to add a hook that integrates with your internal 
+     * of when you'd do this is to add a hook that integrates with your internal
      * monitoring system. These hooks are instantiated using the zero-arg constructor.
      */
     public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks";
@@ -647,7 +654,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator;
 
     /**
-     * The maximum number of messages to batch from the thread receiving off the network to the 
+     * The maximum number of messages to batch from the thread receiving off the network to the
      * executor queues. Must be a power of 2.
      */
     public static final String TOPOLOGY_RECEIVER_BUFFER_SIZE="topology.receiver.buffer.size";
@@ -681,14 +688,14 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class;
 
    /**
-    * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed 
+    * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
     * via the TopologyContext.
     */
     public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size";
     public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = Number.class;
 
     /**
-     * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, 
+     * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
      * an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
      * reported to Zookeeper per task for every 10 second interval of time.
      */
@@ -757,9 +764,9 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
-     * for the java.library.path value. java.library.path tells the JVM where 
+     * for the java.library.path value. java.library.path tells the JVM where
      * to look for native libraries. It is necessary to set this config correctly since
-     * Storm uses the ZeroMQ and JZMQ native libs. 
+     * Storm uses the ZeroMQ and JZMQ native libs.
      */
     public static final String JAVA_LIBRARY_PATH = "java.library.path";
     public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class;
@@ -781,17 +788,17 @@ public class Config extends HashMap<String, Object> {
 
     public static void setDebug(Map conf, boolean isOn) {
         conf.put(Config.TOPOLOGY_DEBUG, isOn);
-    } 
+    }
 
     public void setDebug(boolean isOn) {
         setDebug(this, isOn);
     }
-    
+
     @Deprecated
     public void setOptimize(boolean isOn) {
         put(Config.TOPOLOGY_OPTIMIZE, isOn);
-    } 
-    
+    }
+
     public static void setNumWorkers(Map conf, int workers) {
         conf.put(Config.TOPOLOGY_WORKERS, workers);
     }
@@ -807,7 +814,7 @@ public class Config extends HashMap<String, Object> {
     public void setNumAckers(int numExecutors) {
         setNumAckers(this, numExecutors);
     }
-    
+
     public static void setMessageTimeoutSecs(Map conf, int secs) {
         conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs);
     }
@@ -815,7 +822,7 @@ public class Config extends HashMap<String, Object> {
     public void setMessageTimeoutSecs(int secs) {
         setMessageTimeoutSecs(this, secs);
     }
-    
+
     public static void registerSerialization(Map conf, Class klass) {
         getRegisteredSerializations(conf).add(klass.getName());
     }
@@ -823,17 +830,17 @@ public class Config extends HashMap<String, Object> {
     public void registerSerialization(Class klass) {
         registerSerialization(this, klass);
     }
-    
+
     public static void registerSerialization(Map conf, Class klass, Class<? extends Serializer> serializerClass) {
         Map<String, String> register = new HashMap<String, String>();
         register.put(klass.getName(), serializerClass.getName());
-        getRegisteredSerializations(conf).add(register);        
+        getRegisteredSerializations(conf).add(register);
     }
 
     public void registerSerialization(Class klass, Class<? extends Serializer> serializerClass) {
         registerSerialization(this, klass, serializerClass);
     }
-    
+
     public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) {
         HashMap m = new HashMap();
         m.put("class", klass.getCanonicalName());
@@ -861,7 +868,7 @@ public class Config extends HashMap<String, Object> {
     public void registerDecorator(Class<? extends IKryoDecorator> klass) {
         registerDecorator(this, klass);
     }
-    
+
     public static void setKryoFactory(Map conf, Class<? extends IKryoFactory> klass) {
         conf.put(Config.TOPOLOGY_KRYO_FACTORY, klass.getName());
     }
@@ -877,7 +884,7 @@ public class Config extends HashMap<String, Object> {
     public void setSkipMissingKryoRegistrations(boolean skip) {
        setSkipMissingKryoRegistrations(this, skip);
     }
-    
+
     public static void setMaxTaskParallelism(Map conf, int max) {
         conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max);
     }
@@ -885,7 +892,7 @@ public class Config extends HashMap<String, Object> {
     public void setMaxTaskParallelism(int max) {
         setMaxTaskParallelism(this, max);
     }
-    
+
     public static void setMaxSpoutPending(Map conf, int max) {
         conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max);
     }
@@ -893,23 +900,23 @@ public class Config extends HashMap<String, Object> {
     public void setMaxSpoutPending(int max) {
         setMaxSpoutPending(this, max);
     }
-    
+
     public static void setStatsSampleRate(Map conf, double rate) {
         conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, rate);
-    }    
+    }
 
     public void setStatsSampleRate(double rate) {
         setStatsSampleRate(this, rate);
-    }    
+    }
 
     public static void setFallBackOnJavaSerialization(Map conf, boolean fallback) {
         conf.put(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION, fallback);
-    }    
+    }
 
     public void setFallBackOnJavaSerialization(boolean fallback) {
         setFallBackOnJavaSerialization(this, fallback);
-    }    
-    
+    }
+
     private static List getRegisteredSerializations(Map conf) {
         List ret;
         if(!conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) {
@@ -920,13 +927,13 @@ public class Config extends HashMap<String, Object> {
         conf.put(Config.TOPOLOGY_KRYO_REGISTER, ret);
         return ret;
     }
-    
+
     private static List getRegisteredDecorators(Map conf) {
         List ret;
         if(!conf.containsKey(Config.TOPOLOGY_KRYO_DECORATORS)) {
             ret = new ArrayList();
         } else {
-            ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS));            
+            ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS));
         }
         conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret);
         return ret;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java b/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java
new file mode 100644
index 0000000..1d6bd1d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java
@@ -0,0 +1,63 @@
+package backtype.storm.multilang;
+
+import java.util.List;
+
+/**
+ * BoltMsg is an object that represents the data sent from a shell component to
+ * a bolt process that implements a multi-language protocol. It is the union of
+ * all data types that a bolt can receive from Storm.
+ *
+ * <p>
+ * BoltMsgs are objects sent to the ISerializer interface, for serialization
+ * according to the wire protocol implemented by the serializer. The BoltMsg
+ * class allows for a decoupling between the serialized representation of the
+ * data and the data itself.
+ * </p>
+ */
+public class BoltMsg {
+    private String id;
+    private String comp;
+    private String stream;
+    private long task;
+    private List<Object> tuple;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getComp() {
+        return comp;
+    }
+
+    public void setComp(String comp) {
+        this.comp = comp;
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    public void setStream(String stream) {
+        this.stream = stream;
+    }
+
+    public long getTask() {
+        return task;
+    }
+
+    public void setTask(long task) {
+        this.task = task;
+    }
+
+    public List<Object> getTuple() {
+        return tuple;
+    }
+
+    public void setTuple(List<Object> tuple) {
+        this.tuple = tuple;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/Emission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Emission.java b/storm-core/src/jvm/backtype/storm/multilang/Emission.java
deleted file mode 100644
index d477d8b..0000000
--- a/storm-core/src/jvm/backtype/storm/multilang/Emission.java
+++ /dev/null
@@ -1,79 +0,0 @@
-package backtype.storm.multilang;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Emission is an object that represents the data sent to a shell component
- * from a process that implements a multi-language protocol. It is the union
- * of all data types that a component can send to Storm.
- *
- * <p> Emissions are objects received from the ISerializer interface, after
- * the serializer has deserialized the data from the underlying wire protocol.
- * The Emission class allows for a decoupling between the serialized
- * representation of the data and the data itself.</p>
- */
-public class Emission {
-	private String command;
-	private String id;
-	private List<String> anchors;
-	private String stream;
-	private long task;
-	private String msg;
-	private List<Object> tuple;
-
-	public String getCommand() {
-		return command;
-	}
-	public void setCommand(String command) {
-		this.command = command;
-	}
-	public String getId() {
-		return id;
-	}
-	public void setId(String id) {
-		this.id = id;
-	}
-	public List<String> getAnchors() {
-		return anchors;
-	}
-	public void setAnchors(List<String> anchors) {
-		this.anchors = anchors;
-	}
-	public void addAnchor(String anchor) {
-		if (anchors == null) {
-			anchors = new ArrayList<String>();
-		}
-		this.anchors.add(anchor);
-	}
-	public String getStream() {
-		return stream;
-	}
-	public void setStream(String stream) {
-		this.stream = stream;
-	}
-	public long getTask() {
-		return task;
-	}
-	public void setTask(long task) {
-		this.task = task;
-	}
-	public String getMsg() {
-		return msg;
-	}
-	public void setMsg(String msg) {
-		this.msg = msg;
-	}
-	public List<Object> getTuple() {
-		return tuple;
-	}
-	public void setTuple(List<Object> tuple) {
-		this.tuple = tuple;
-	}
-	public void addTuple(Object tuple) {
-		if (this.tuple == null) {
-			this.tuple = new ArrayList<Object>();
-		}
-		this.tuple.add(tuple);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
index 691b432..a6ccfd4 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
@@ -10,15 +10,56 @@ import java.util.Map;
 import backtype.storm.task.TopologyContext;
 
 /**
- * The ISerializer interface describes the methods that an object should implement
- * to provide serialization and de-serialization capabilities to non-JVM
- * language components.
+ * The ISerializer interface describes the methods that an object should
+ * implement to provide serialization and de-serialization capabilities to
+ * non-JVM language components.
  */
 public interface ISerializer extends Serializable {
-	void initialize (OutputStream processIn, InputStream processOut);
-	Number connect (Map conf, TopologyContext context) throws IOException, NoOutputException;
-	Emission readEmission() throws IOException, NoOutputException;
-	void writeImmission(Immission immission) throws IOException;
-	void writeSpoutMsg(SpoutMsg msg) throws IOException;
-	void writeTaskIds(List<Integer> taskIds) throws IOException;
+
+    /**
+     * This method sets the input and output streams of the serializer
+     *
+     * @param processIn output stream to non-JVM component
+     * @param processOut input stream from non-JVM component
+     */
+    void initialize(OutputStream processIn, InputStream processOut);
+
+    /**
+     * This method transmits the Storm config to the non-JVM process and
+     * receives its pid.
+     *
+     * @param conf storm configuration
+     * @param context topology context
+     * @return process pid
+     */
+    Number connect(Map conf, TopologyContext context) throws IOException,
+            NoOutputException;
+
+    /**
+     * This method receives a shell message from the non-JVM process
+     *
+     * @return shell message
+     */
+    ShellMsg readShellMsg() throws IOException, NoOutputException;
+
+    /**
+     * This method sends a bolt message to a non-JVM bolt process
+     *
+     * @param msg bolt message
+     */
+    void writeBoltMsg(BoltMsg msg) throws IOException;
+
+    /**
+     * This method sends a spout message to a non-JVM spout process
+     *
+     * @param msg spout message
+     */
+    void writeSpoutMsg(SpoutMsg msg) throws IOException;
+
+    /**
+     * This method sends a list of task IDs to a non-JVM bolt process
+     *
+     * @param taskIds list of task IDs
+     */
+    void writeTaskIds(List<Integer> taskIds) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/Immission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Immission.java b/storm-core/src/jvm/backtype/storm/multilang/Immission.java
deleted file mode 100644
index 968f728..0000000
--- a/storm-core/src/jvm/backtype/storm/multilang/Immission.java
+++ /dev/null
@@ -1,52 +0,0 @@
-package backtype.storm.multilang;
-
-import java.util.List;
-
-/**
- * Immission is an object that represents the data sent from a shell component
- * to a process that implements a multi-language protocol. It is the union
- * of all data types that a component can receive from Storm.
- *
- * <p> Immissions are objects sent to the ISerializer interface, for
- * serialization according to the wire protocol implemented by the serializer.
- * The Immission class allows for a decoupling between the serialized
- * representation of the data and the data itself.</p>
- */
-public class Immission {
-	private String id;
-	private String comp;
-	private String stream;
-	private long task;
-	private List<Object> tuple;
-
-	public String getId() {
-		return id;
-	}
-	public void setId(String id) {
-		this.id = id;
-	}
-	public String getComp() {
-		return comp;
-	}
-	public void setComp(String comp) {
-		this.comp = comp;
-	}
-	public String getStream() {
-		return stream;
-	}
-	public void setStream(String stream) {
-		this.stream = stream;
-	}
-	public long getTask() {
-		return task;
-	}
-	public void setTask(long task) {
-		this.task = task;
-	}
-	public List<Object> getTuple() {
-		return tuple;
-	}
-	public void setTuple(List<Object> tuple) {
-		this.tuple = tuple;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
index ba574e5..ceb8cb3 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -22,126 +22,128 @@ import backtype.storm.utils.Utils;
  * JsonSerializer implements the JSON multilang protocol.
  */
 public class JsonSerializer implements ISerializer {
-	private DataOutputStream processIn;
+    private DataOutputStream processIn;
     private BufferedReader processOut;
 
-	public void initialize (OutputStream processIn, InputStream processOut) {
-		this.processIn = new DataOutputStream(processIn);;
-		this.processOut = new BufferedReader(new InputStreamReader(processOut));
-	}
+    public void initialize(OutputStream processIn, InputStream processOut) {
+        this.processIn = new DataOutputStream(processIn);
+        this.processOut = new BufferedReader(new InputStreamReader(processOut));
+    }
 
-	public Number connect (Map conf, TopologyContext context) throws IOException, NoOutputException {
-		JSONObject setupInfo = new JSONObject();
+    public Number connect(Map conf, TopologyContext context)
+            throws IOException, NoOutputException {
+        JSONObject setupInfo = new JSONObject();
         setupInfo.put("pidDir", context.getPIDDir());
         setupInfo.put("conf", conf);
         setupInfo.put("context", context);
-    	writeMessage(setupInfo);
+        writeMessage(setupInfo);
 
-    	Number pid = (Number)((JSONObject)readMessage()).get("pid");
+        Number pid = (Number) ((JSONObject) readMessage()).get("pid");
         return pid;
-	}
-
-	public void writeImmission(Immission immission) throws IOException {
-		JSONObject obj = new JSONObject();
-        obj.put("id", immission.getId());
-        obj.put("comp", immission.getComp());
-        obj.put("stream", immission.getStream());
-        obj.put("task", immission.getTask());
-        obj.put("tuple", immission.getTuple());
+    }
+
+    public void writeBoltMsg(BoltMsg boltMsg) throws IOException {
+        JSONObject obj = new JSONObject();
+        obj.put("id", boltMsg.getId());
+        obj.put("comp", boltMsg.getComp());
+        obj.put("stream", boltMsg.getStream());
+        obj.put("task", boltMsg.getTask());
+        obj.put("tuple", boltMsg.getTuple());
         writeMessage(obj);
-	}
+    }
 
-	public void writeSpoutMsg(SpoutMsg msg) throws IOException {
-		JSONObject obj = new JSONObject();
+    public void writeSpoutMsg(SpoutMsg msg) throws IOException {
+        JSONObject obj = new JSONObject();
         obj.put("command", msg.getCommand());
         obj.put("id", msg.getId());
         writeMessage(obj);
-	}
+    }
 
-	public void writeTaskIds(List<Integer> taskIds) throws IOException {
+    public void writeTaskIds(List<Integer> taskIds) throws IOException {
         writeMessage(taskIds);
-	}
+    }
 
-	private void writeMessage(Object msg) throws IOException {
+    private void writeMessage(Object msg) throws IOException {
         writeString(JSONValue.toJSONString(msg));
     }
 
-	private void writeString(String str) throws IOException {
+    private void writeString(String str) throws IOException {
         byte[] strBytes = str.getBytes("UTF-8");
         processIn.write(strBytes, 0, strBytes.length);
         processIn.writeBytes("\nend\n");
         processIn.flush();
     }
 
-	public Emission readEmission() throws IOException, NoOutputException {
-        JSONObject msg = (JSONObject)readMessage();
-    	Emission emission = new Emission();
+    public ShellMsg readShellMsg() throws IOException, NoOutputException {
+        JSONObject msg = (JSONObject) readMessage();
+        ShellMsg shellMsg = new ShellMsg();
 
-    	String command = (String) msg.get("command");
-        emission.setCommand(command);
+        String command = (String) msg.get("command");
+        shellMsg.setCommand(command);
 
-    	String stream = (String) msg.get("stream");
-        if(stream == null) stream = Utils.DEFAULT_STREAM_ID;
-        emission.setStream(stream);
+        String stream = (String) msg.get("stream");
+        if (stream == null)
+            stream = Utils.DEFAULT_STREAM_ID;
+        shellMsg.setStream(stream);
 
         Object taskObj = msg.get("task");
         if (taskObj != null) {
-        	emission.setTask((Long) taskObj);
+            shellMsg.setTask((Long) taskObj);
         }
 
-        emission.setTuple((List) msg.get("tuple"));
+        shellMsg.setTuple((List) msg.get("tuple"));
 
         List<Tuple> anchors = new ArrayList<Tuple>();
         Object anchorObj = msg.get("anchors");
-        if(anchorObj!=null) {
-            if(anchorObj instanceof String) {
+        if (anchorObj != null) {
+            if (anchorObj instanceof String) {
                 anchorObj = Arrays.asList(anchorObj);
             }
-            for(Object o: (List) anchorObj) {
-                emission.addAnchor((String)o);
+            for (Object o : (List) anchorObj) {
+                shellMsg.addAnchor((String) o);
             }
         }
 
-        return emission;
+        return shellMsg;
     }
 
-	private Object readMessage() throws IOException, NoOutputException {
-		String string = readString();
+    private Object readMessage() throws IOException, NoOutputException {
+        String string = readString();
         Object msg = JSONValue.parse(string);
         if (msg != null) {
-        	return msg;
+            return msg;
         } else {
-        	throw new IOException("unable to parse: " + string);
+            throw new IOException("unable to parse: " + string);
         }
-	}
+    }
 
     private String readString() throws IOException, NoOutputException {
         StringBuilder line = new StringBuilder();
 
-        //synchronized (processOut) {
-            while (true) {
-                String subline = processOut.readLine();
-                if(subline==null) {
-                    StringBuilder errorMessage = new StringBuilder();
-                    errorMessage.append("Pipe to subprocess seems to be broken!");
-                    if (line.length() == 0) {
-                        errorMessage.append(" No output read.\n");
-                    }
-                    else {
-                        errorMessage.append(" Currently read output: " + line.toString() + "\n");
-                    }
-                    errorMessage.append("Serializer Exception:\n");
-                    throw new NoOutputException(errorMessage.toString());
-                }
-                if(subline.equals("end")) {
-                    break;
-                }
-                if(line.length()!=0) {
-                    line.append("\n");
+        // synchronized (processOut) {
+        while (true) {
+            String subline = processOut.readLine();
+            if (subline == null) {
+                StringBuilder errorMessage = new StringBuilder();
+                errorMessage.append("Pipe to subprocess seems to be broken!");
+                if (line.length() == 0) {
+                    errorMessage.append(" No output read.\n");
+                } else {
+                    errorMessage.append(" Currently read output: "
+                            + line.toString() + "\n");
                 }
-                line.append(subline);
+                errorMessage.append("Serializer Exception:\n");
+                throw new NoOutputException(errorMessage.toString());
             }
-        //}
+            if (subline.equals("end")) {
+                break;
+            }
+            if (line.length() != 0) {
+                line.append("\n");
+            }
+            line.append(subline);
+        }
+        // }
         return line.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
index 8ffb7dd..fd5ef72 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
@@ -5,8 +5,19 @@ package backtype.storm.multilang;
  * non-JVM process.
  */
 public class NoOutputException extends Exception {
-	public NoOutputException() { super(); }
-	public NoOutputException(String message) { super(message); }
-	public NoOutputException(String message, Throwable cause) { super(message, cause); }
-	public NoOutputException(Throwable cause) { super(cause); }
+    public NoOutputException() {
+        super();
+    }
+
+    public NoOutputException(String message) {
+        super(message);
+    }
+
+    public NoOutputException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NoOutputException(Throwable cause) {
+        super(cause);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
new file mode 100644
index 0000000..d56ba78
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
@@ -0,0 +1,96 @@
+package backtype.storm.multilang;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * ShellMsg is an object that represents the data sent to a shell component from
+ * a process that implements a multi-language protocol. It is the union of all
+ * data types that a component can send to Storm.
+ *
+ * <p>
+ * ShellMsgs are objects received from the ISerializer interface, after the
+ * serializer has deserialized the data from the underlying wire protocol. The
+ * ShellMsg class allows for a decoupling between the serialized representation
+ * of the data and the data itself.
+ * </p>
+ */
+public class ShellMsg {
+    private String command;
+    private String id;
+    private List<String> anchors;
+    private String stream;
+    private long task;
+    private String msg;
+    private List<Object> tuple;
+
+    public String getCommand() {
+        return command;
+    }
+
+    public void setCommand(String command) {
+        this.command = command;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public List<String> getAnchors() {
+        return anchors;
+    }
+
+    public void setAnchors(List<String> anchors) {
+        this.anchors = anchors;
+    }
+
+    public void addAnchor(String anchor) {
+        if (anchors == null) {
+            anchors = new ArrayList<String>();
+        }
+        this.anchors.add(anchor);
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    public void setStream(String stream) {
+        this.stream = stream;
+    }
+
+    public long getTask() {
+        return task;
+    }
+
+    public void setTask(long task) {
+        this.task = task;
+    }
+
+    public String getMsg() {
+        return msg;
+    }
+
+    public void setMsg(String msg) {
+        this.msg = msg;
+    }
+
+    public List<Object> getTuple() {
+        return tuple;
+    }
+
+    public void setTuple(List<Object> tuple) {
+        this.tuple = tuple;
+    }
+
+    public void addTuple(Object tuple) {
+        if (this.tuple == null) {
+            this.tuple = new ArrayList<Object>();
+        }
+        this.tuple.add(tuple);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
index c8063eb..9b5ffce 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
@@ -1,28 +1,34 @@
 package backtype.storm.multilang;
 
 /**
- * SpoutMsg is an object that represents the data sent from a shell spout
- * to a process that implements a multi-language spout. The SpoutMsg is used
- * to send a "next", "ack" or "fail" message to a spout.
+ * SpoutMsg is an object that represents the data sent from a shell spout to a
+ * process that implements a multi-language spout. The SpoutMsg is used to send
+ * a "next", "ack" or "fail" message to a spout.
  *
- * <p> Spout messages are objects sent to the ISerializer interface, for
+ * <p>
+ * Spout messages are objects sent to the ISerializer interface, for
  * serialization according to the wire protocol implemented by the serializer.
  * The SpoutMsg class allows for a decoupling between the serialized
- * representation of the data and the data itself.</p>
+ * representation of the data and the data itself.
+ * </p>
  */
 public class SpoutMsg {
-	private String command;
-	private String id;
-	public String getCommand() {
-		return command;
-	}
-	public void setCommand(String command) {
-		this.command = command;
-	}
-	public String getId() {
-		return id;
-	}
-	public void setId(String id) {
-		this.id = id;
-	}
+    private String command;
+    private String id;
+
+    public String getCommand() {
+        return command;
+    }
+
+    public void setCommand(String command) {
+        this.command = command;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index c9ef682..72cebe2 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -1,13 +1,10 @@
 package backtype.storm.spout;
 
 import backtype.storm.generated.ShellComponent;
-import backtype.storm.multilang.Emission;
-import backtype.storm.multilang.ISerializer;
-import backtype.storm.multilang.JsonSerializer;
+import backtype.storm.multilang.ShellMsg;
 import backtype.storm.multilang.SpoutMsg;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.utils.ShellProcess;
-import backtype.storm.utils.Utils;
 import java.util.Map;
 import java.util.List;
 import java.io.IOException;
@@ -25,22 +22,16 @@ public class ShellSpout implements ISpout {
 
     public ShellSpout(ShellComponent component) {
         this(component.get_execution_command(), component.get_script());
-        _process = new ShellProcess(new JsonSerializer(), _command);
     }
 
     public ShellSpout(String... command) {
         _command = command;
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellSpout(ISerializer serializer, String... command) {
-        _command = command;
-        _process = new ShellProcess(serializer, _command);
     }
 
     public void open(Map stormConf, TopologyContext context,
                      SpoutOutputCollector collector) {
         _collector = collector;
+        _process = new ShellProcess(_command);
 
         Number subpid = _process.launch(stormConf, context);
         LOG.info("Launched subprocess with pid " + subpid);
@@ -82,23 +73,24 @@ public class ShellSpout implements ISpout {
             _process.writeSpoutMsg(spoutMsg);
 
             while (true) {
-                Emission emission = _process.readEmission();
-                String command = emission.getCommand();
+                ShellMsg shellMsg = _process.readShellMsg();
+                String command = shellMsg.getCommand();
                 if (command.equals("sync")) {
                     return;
                 } else if (command.equals("log")) {
-                    String msg = emission.getMsg();
+                    String msg = shellMsg.getMsg();
                     LOG.info("Shell msg: " + msg);
                 } else if (command.equals("emit")) {
-                    String stream = emission.getStream();
-                    Long task = emission.getTask();
-                    List<Object> tuple = emission.getTuple();
-                    Object messageId = emission.getId();
+                    String stream = shellMsg.getStream();
+                    Long task = shellMsg.getTask();
+                    List<Object> tuple = shellMsg.getTuple();
+                    Object messageId = shellMsg.getId();
                     if (task == 0) {
                         List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
                         _process.writeTaskIds(outtasks);
                     } else {
-                        _collector.emitDirect((int)task.longValue(), stream, tuple, messageId);
+                        _collector.emitDirect((int) task.longValue(), stream,
+                                tuple, messageId);
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index 36585e3..6a6c305 100644
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -3,10 +3,9 @@ package backtype.storm.task;
 import backtype.storm.generated.ShellComponent;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.utils.ShellProcess;
-import backtype.storm.multilang.Emission;
-import backtype.storm.multilang.ISerializer;
-import backtype.storm.multilang.Immission;
-import backtype.storm.multilang.JsonSerializer;
+import backtype.storm.multilang.BoltMsg;
+import backtype.storm.multilang.ShellMsg;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
@@ -62,23 +61,17 @@ public class ShellBolt implements IBolt {
 
     public ShellBolt(ShellComponent component) {
         this(component.get_execution_command(), component.get_script());
-        _process = new ShellProcess(new JsonSerializer(), _command);
     }
 
     public ShellBolt(String... command) {
         _command = command;
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellBolt(ISerializer serializer, String... command) {
-        _command = command;
-        _process = new ShellProcess(serializer, _command);
     }
 
     public void prepare(Map stormConf, TopologyContext context,
                         final OutputCollector collector) {
         _rand = new Random();
         _collector = collector;
+        _process = new ShellProcess(_command);
 
         //subprocesses must send their pid first thing
         Number subpid = _process.launch(stormConf, context);
@@ -89,20 +82,20 @@ public class ShellBolt implements IBolt {
             public void run() {
                 while (_running) {
                     try {
-                        Emission emission = _process.readEmission();
+                        ShellMsg shellMsg = _process.readShellMsg();
 
-                        String command = emission.getCommand();
+                        String command = shellMsg.getCommand();
                         if(command.equals("ack")) {
-                            handleAck(emission.getId());
+                            handleAck(shellMsg.getId());
                         } else if (command.equals("fail")) {
-                            handleFail(emission.getId());
+                            handleFail(shellMsg.getId());
                         } else if (command.equals("error")) {
-                            handleError(emission.getMsg());
+                            handleError(shellMsg.getMsg());
                         } else if (command.equals("log")) {
-                            String msg = emission.getMsg();
+                            String msg = shellMsg.getMsg();
                             LOG.info("Shell msg: " + msg);
                         } else if (command.equals("emit")) {
-                            handleEmit(emission);
+                            handleEmit(shellMsg);
                         }
                     } catch (InterruptedException e) {
                     } catch (Throwable t) {
@@ -118,12 +111,13 @@ public class ShellBolt implements IBolt {
             public void run() {
                 while (_running) {
                     try {
-                    	// FIXME: This can either be TaskIds or Immissions
                         Object write = _pendingWrites.poll(1, SECONDS);
-                        if (write instanceof Immission) {
-                            _process.writeImmission((Immission)write);
+                        if (write instanceof BoltMsg) {
+                            _process.writeBoltMsg((BoltMsg)write);
                         } else if (write instanceof List<?>) {
-
+                            _process.writeTaskIds((List<Integer>)write);
+                        } else {
+                            throw new RuntimeException("Cannot write object to bolt:\n" + write.toString());
                         }
                     } catch (InterruptedException e) {
                     } catch (Throwable t) {
@@ -145,14 +139,14 @@ public class ShellBolt implements IBolt {
         String genId = Long.toString(_rand.nextLong());
         _inputs.put(genId, input);
         try {
-            Immission immission = new Immission();
-            immission.setId(genId);
-            immission.setComp(input.getSourceComponent());
-            immission.setStream(input.getSourceStreamId());
-            immission.setTask(input.getSourceTask());
-            immission.setTuple(input.getValues());
-
-            _pendingWrites.put(immission);
+            BoltMsg boltMsg = new BoltMsg();
+            boltMsg.setId(genId);
+            boltMsg.setComp(input.getSourceComponent());
+            boltMsg.setStream(input.getSourceStreamId());
+            boltMsg.setTask(input.getSourceTask());
+            boltMsg.setTuple(input.getValues());
+
+            _pendingWrites.put(boltMsg);
         } catch(InterruptedException e) {
             throw new RuntimeException("Error during multilang processing", e);
         }
@@ -184,9 +178,9 @@ public class ShellBolt implements IBolt {
         _collector.reportError(new Exception("Shell Process Exception: " + msg));
     }
 
-    private void handleEmit(Emission emission) throws InterruptedException {
+    private void handleEmit(ShellMsg shellMsg) throws InterruptedException {
     	List<Tuple> anchors = new ArrayList<Tuple>();
-    	for (String anchor : emission.getAnchors()) {
+    	for (String anchor : shellMsg.getAnchors()) {
 	    	Tuple t = _inputs.get(anchor);
 	        if (t == null) {
 	            throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
@@ -194,11 +188,12 @@ public class ShellBolt implements IBolt {
 	        anchors.add(t);
     	}
 
-        if(emission.getTask() == 0) {
-            List<Integer> outtasks = _collector.emit(emission.getStream(), anchors, emission.getTuple());
+        if(shellMsg.getTask() == 0) {
+            List<Integer> outtasks = _collector.emit(shellMsg.getStream(), anchors, shellMsg.getTuple());
             _pendingWrites.put(outtasks);
         } else {
-            _collector.emitDirect((int)emission.getTask(), emission.getStream(), anchors, emission.getTuple());
+            _collector.emitDirect((int) shellMsg.getTask(),
+                    shellMsg.getStream(), anchors, shellMsg.getTuple());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/ee0678c7/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
index 38976bc..8871a77 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
@@ -1,9 +1,10 @@
 package backtype.storm.utils;
 
-import backtype.storm.multilang.Emission;
+import backtype.storm.Config;
 import backtype.storm.multilang.ISerializer;
-import backtype.storm.multilang.Immission;
+import backtype.storm.multilang.BoltMsg;
 import backtype.storm.multilang.NoOutputException;
+import backtype.storm.multilang.ShellMsg;
 import backtype.storm.multilang.SpoutMsg;
 import backtype.storm.task.TopologyContext;
 import java.io.File;
@@ -18,85 +19,109 @@ import org.apache.log4j.Logger;
 
 public class ShellProcess implements Serializable {
     public static Logger LOG = Logger.getLogger(ShellProcess.class);
-    private Process _subprocess;
-    private InputStream processErrorStream;
-    private String[] command;
-    public ISerializer serializer;
+    private Process      _subprocess;
+    private InputStream  processErrorStream;
+    private String[]     command;
+    public ISerializer   serializer;
 
-    public ShellProcess(ISerializer serializer, String[] command) {
+    public ShellProcess(String[] command) {
         this.command = command;
-        this.serializer = serializer;
     }
 
     public Number launch(Map conf, TopologyContext context) {
         ProcessBuilder builder = new ProcessBuilder(command);
         builder.directory(new File(context.getCodeDir()));
 
+        this.serializer = getSerializer(conf);
+
         Number pid;
         try {
-	        _subprocess = builder.start();
-	        processErrorStream = _subprocess.getErrorStream();
-	        serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream());
-	        pid = serializer.connect(conf, context);
+            _subprocess = builder.start();
+            processErrorStream = _subprocess.getErrorStream();
+            serializer.initialize(_subprocess.getOutputStream(),
+                    _subprocess.getInputStream());
+            pid = serializer.connect(conf, context);
         } catch (IOException e) {
-        	throw new RuntimeException("Error when launching multilang subprocess\n" + getErrorsString(), e);
+            throw new RuntimeException(
+                    "Error when launching multilang subprocess\n"
+                            + getErrorsString(), e);
         } catch (NoOutputException e) {
             throw new RuntimeException(e + getErrorsString() + "\n");
-    	}
+        }
         return pid;
     }
 
+    private ISerializer getSerializer(Map conf) {
+        //get factory class name
+        String serializer_className = (String)conf.get(Config.STORM_MULTILANG_SERIALIZER);
+        LOG.info("Storm multilang serializer:" + serializer_className);
+
+        ISerializer serializer = null;
+        try {
+            //create a factory class
+            Class klass = Class.forName(serializer_className);
+            //obtain a serializer object
+            Object obj = klass.newInstance();
+            serializer = (ISerializer)obj;
+        } catch(Exception e) {
+            throw new RuntimeException("Failed to construct multilang serializer from serializer " + serializer_className, e);
+        }
+        return serializer;
+    }
+
     public void destroy() {
         _subprocess.destroy();
     }
 
-    public Emission readEmission() throws IOException {
-    	try  {
-    		return serializer.readEmission();
-    	} catch (NoOutputException e) {
+    public ShellMsg readShellMsg() throws IOException {
+        try {
+            return serializer.readShellMsg();
+        } catch (NoOutputException e) {
             throw new RuntimeException(e + getErrorsString() + "\n");
-    	}
+        }
     }
 
-    public void writeImmission(Immission msg) throws IOException {
-    	serializer.writeImmission(msg);
-        // drain the error stream to avoid dead lock because of full error stream buffer
+    public void writeBoltMsg(BoltMsg msg) throws IOException {
+        serializer.writeBoltMsg(msg);
+        // drain the error stream to avoid dead lock because of full error
+        // stream buffer
         drainErrorStream();
     }
 
     public void writeSpoutMsg(SpoutMsg msg) throws IOException {
-    	serializer.writeSpoutMsg(msg);
-        // drain the error stream to avoid dead lock because of full error stream buffer
+        serializer.writeSpoutMsg(msg);
+        // drain the error stream to avoid dead lock because of full error
+        // stream buffer
         drainErrorStream();
     }
 
     public void writeTaskIds(List<Integer> taskIds) throws IOException {
-    	serializer.writeTaskIds(taskIds);
-        // drain the error stream to avoid dead lock because of full error stream buffer
+        serializer.writeTaskIds(taskIds);
+        // drain the error stream to avoid dead lock because of full error
+        // stream buffer
         drainErrorStream();
     }
 
-    public void drainErrorStream()
-    {
+    public void drainErrorStream() {
         try {
-            while (processErrorStream.available() > 0)
-            {
+            while (processErrorStream.available() > 0) {
                 int bufferSize = processErrorStream.available();
-                byte[] errorReadingBuffer =  new byte[bufferSize];
+                byte[] errorReadingBuffer = new byte[bufferSize];
 
                 processErrorStream.read(errorReadingBuffer, 0, bufferSize);
 
-                LOG.info("Got error from shell process: " + new String(errorReadingBuffer));
+                LOG.info("Got error from shell process: "
+                        + new String(errorReadingBuffer));
             }
-        } catch(Exception e) {
+        } catch (Exception e) {
         }
     }
 
     public String getErrorsString() {
-        if(processErrorStream!=null) {
+        if (processErrorStream != null) {
             try {
                 return IOUtils.toString(processErrorStream);
-            } catch(IOException e) {
+            } catch (IOException e) {
                 return "(Unable to capture error stream)";
             }
         } else {


[20/24] git commit: Made the multilang serialiser topology specific

Posted by bo...@apache.org.
Made the multilang serialiser topology specific


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

Branch: refs/heads/master
Commit: 660d8c42ee8876847209e57ffa2c200ceefbbbbd
Parents: f5f41a0
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Thu May 1 12:29:48 2014 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Thu May 1 12:29:48 2014 +0200

----------------------------------------------------------------------
 conf/defaults.yaml                                     |  2 +-
 storm-core/src/jvm/backtype/storm/Config.java          | 13 ++++++-------
 .../src/jvm/backtype/storm/utils/ShellProcess.java     |  4 ++--
 3 files changed, 9 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/660d8c42/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 66b2d41..c7af199 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -35,7 +35,6 @@ storm.zookeeper.retry.intervalceiling.millis: 30000
 storm.cluster.mode: "distributed" # can be distributed or local
 storm.local.mode.zmq: false
 storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
-storm.multilang.serializer: "backtype.storm.multilang.JsonSerializer"
 storm.messaging.transport: "backtype.storm.messaging.netty.Context"
 
 ### nimbus.* configs are for the master
@@ -119,6 +118,7 @@ topology.acker.executors: null
 topology.tasks: null
 # maximum amount of time a message has to complete before it's considered failed
 topology.message.timeout.secs: 30
+topology.multilang.serializer: "backtype.storm.multilang.JsonSerializer"
 topology.skip.missing.kryo.registrations: false
 topology.max.task.parallelism: null
 topology.max.spout.pending: null

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/660d8c42/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 d99325c..593898f 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -44,13 +44,6 @@ import java.util.Map;
  */
 public class Config extends HashMap<String, Object> {
     /**
-     * The serializer for communication between shell components and non-JVM
-     * processes
-     */
-    public static final String STORM_MULTILANG_SERIALIZER = "storm.multilang.serializer";
-    public static final Object STORM_MULTILANG_SERIALIZER_SCHEMA = String.class;
-
-    /**
      * The transporter for communication among Storm tasks
      */
     public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport";
@@ -506,6 +499,12 @@ public class Config extends HashMap<String, Object> {
     public static final String TOPOLOGY_DEBUG = "topology.debug";
     public static final Object TOPOLOGY_DEBUG_SCHEMA = Boolean.class;
 
+    /**
+     * The serializer for communication between shell components and non-JVM
+     * processes
+     */
+    public static final String TOPOLOGY_MULTILANG_SERIALIZER = "topology.multilang.serializer";
+    public static final Object TOPOLOGY_MULTILANG_SERIALIZER_SCHEMA = String.class;
 
     /**
      * Whether or not the master should optimize topologies by running multiple

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/660d8c42/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
index 35a8c94..7455c9b 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
@@ -72,8 +72,8 @@ public class ShellProcess implements Serializable {
 
     private ISerializer getSerializer(Map conf) {
         //get factory class name
-        String serializer_className = (String)conf.get(Config.STORM_MULTILANG_SERIALIZER);
-        LOG.info("Storm multilang serializer:" + serializer_className);
+        String serializer_className = (String)conf.get(Config.TOPOLOGY_MULTILANG_SERIALIZER);
+        LOG.info("Storm multilang serializer: " + serializer_className);
 
         ISerializer serializer = null;
         try {


[06/24] git commit: Merge branch 'master' of github.com:jsgilmore/storm

Posted by bo...@apache.org.
Merge branch 'master' of github.com:jsgilmore/storm


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

Branch: refs/heads/master
Commit: 06aad3f99d82bcc29c4a827e892d48aaf25e2fd6
Parents: ee0678c 3219a4f
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Tue Oct 8 13:53:59 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Tue Oct 8 13:53:59 2013 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[13/24] git commit: Added some trailing whitespace to Config.java

Posted by bo...@apache.org.
Added some trailing whitespace to Config.java


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

Branch: refs/heads/master
Commit: dfbd045966a9a909aaa0e7e31eac59639a86c3c9
Parents: 16ba0c7
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Thu Oct 10 14:58:51 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Thu Oct 10 14:58:51 2013 +0200

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/Config.java | 122 ++++++++++-----------
 1 file changed, 61 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/dfbd0459/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 f94f376..f8f687c 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -10,11 +10,11 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Topology configs are specified as a plain old map. This class provides a
- * convenient way to create a topology config map by providing setter methods for
- * all the configs that can be set. It also makes it easier to do things like add
+ * Topology configs are specified as a plain old map. This class provides a 
+ * convenient way to create a topology config map by providing setter methods for 
+ * all the configs that can be set. It also makes it easier to do things like add 
  * serializations.
- *
+ * 
  * <p>This class also provides constants for all the configurations possible on
  * a Storm cluster and Storm topology. Each constant is paired with a schema
  * that defines the validity criterion of the corresponding field. Default
@@ -22,7 +22,7 @@ import java.util.Map;
  *
  * <p>Note that you may put other configurations in any of the configs. Storm
  * will ignore anything it doesn't recognize, but your topologies are free to make
- * use of them by reading them in the prepare method of Bolts or the open method of
+ * use of them by reading them in the prepare method of Bolts or the open method of 
  * Spouts.</p>
  */
 public class Config extends HashMap<String, Object> {
@@ -42,25 +42,25 @@ public class Config extends HashMap<String, Object> {
     /**
      * Netty based messaging: The buffer size for send/recv buffer
      */
-    public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size";
+    public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; 
     public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = Number.class;
 
     /**
      * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible
      */
-    public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries";
+    public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; 
     public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = Number.class;
 
     /**
-     * Netty based messaging: The min # of milliseconds that a peer will wait.
+     * Netty based messaging: The min # of milliseconds that a peer will wait. 
      */
-    public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms";
+    public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; 
     public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = Number.class;
 
     /**
-     * Netty based messaging: The max # of milliseconds that a peer will wait.
+     * Netty based messaging: The max # of milliseconds that a peer will wait. 
      */
-    public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms";
+    public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; 
     public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = Number.class;
 
     /**
@@ -97,7 +97,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A global task scheduler used to assign topologies's tasks to supervisors' wokers.
-     *
+     * 
      * If this is not set, a default system scheduler will be used.
      */
     public static final String STORM_SCHEDULER = "storm.scheduler";
@@ -110,9 +110,9 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class;
 
     /**
-     * The hostname the supervisors/workers should report to nimbus. If unset, Storm will
+     * The hostname the supervisors/workers should report to nimbus. If unset, Storm will 
      * get the hostname to report by calling <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
-     *
+     * 
      * You should set this config when you dont have a DNS which supervisors/workers
      * can utilize to find each other based on hostname got from calls to
      * <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
@@ -127,16 +127,16 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
-     * The serializer class for ListDelegate (tuple payload).
+     * The serializer class for ListDelegate (tuple payload). 
      * The default serializer will be ListDelegateSerializer
      */
     public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer";
     public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class;
 
     /**
-     * Whether or not to use ZeroMQ for messaging in local mode. If this is set
-     * to false, then Storm will use a pure-Java messaging system. The purpose
-     * of this flag is to make it easy to run Storm in local mode by eliminating
+     * Whether or not to use ZeroMQ for messaging in local mode. If this is set 
+     * to false, then Storm will use a pure-Java messaging system. The purpose 
+     * of this flag is to make it easy to run Storm in local mode by eliminating 
      * the need for native dependencies, which can be difficult to install.
      *
      * Defaults to false.
@@ -274,7 +274,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = Number.class;
 
     /**
-     * Whether or not nimbus should reassign tasks if it detects that a task goes down.
+     * Whether or not nimbus should reassign tasks if it detects that a task goes down. 
      * Defaults to true, and it's not recommended to change this value.
      */
     public static final String NIMBUS_REASSIGN = "nimbus.reassign";
@@ -338,19 +338,19 @@ public class Config extends HashMap<String, Object> {
     public static final Object DRPC_PORT_SCHEMA = Number.class;
 
     /**
-     * DRPC thrift server worker threads
+     * DRPC thrift server worker threads 
      */
     public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
     public static final Object DRPC_WORKER_THREADS_SCHEMA = Number.class;
 
     /**
-     * DRPC thrift server queue size
+     * DRPC thrift server queue size 
      */
     public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
     public static final Object DRPC_QUEUE_SIZE_SCHEMA = Number.class;
 
     /**
-     * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.
+     * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back. 
      */
     public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
     public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = Number.class;
@@ -498,8 +498,8 @@ public class Config extends HashMap<String, Object> {
     /**
      * How many instances to create for a spout/bolt. A task runs on a thread with zero or more
      * other tasks for the same spout/bolt. The number of tasks for a spout/bolt is always
-     * the same throughout the lifetime of a topology, but the number of executors (threads) for
-     * a spout/bolt can change over time. This allows a topology to scale to more or less resources
+     * the same throughout the lifetime of a topology, but the number of executors (threads) for 
+     * a spout/bolt can change over time. This allows a topology to scale to more or less resources 
      * without redeploying the topology or violating the constraints of Storm (such as a fields grouping
      * guaranteeing that the same value goes to the same task).
      */
@@ -538,8 +538,8 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A list of classes that customize storm's kryo instance during start-up.
-     * Each listed class name must implement IKryoDecorator. During start-up the
-     * listed class is instantiated with 0 arguments, then its 'decorate' method
+     * Each listed class name must implement IKryoDecorator. During start-up the 
+     * listed class is instantiated with 0 arguments, then its 'decorate' method 
      * is called with storm's kryo instance as the only argument.
      */
     public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators";
@@ -569,7 +569,7 @@ public class Config extends HashMap<String, Object> {
 
     /*
      * A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format).
-     * Each listed class will be routed all the metrics data generated by the storm metrics API.
+     * Each listed class will be routed all the metrics data generated by the storm metrics API. 
      * Each listed class maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable.
      */
     public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register";
@@ -585,24 +585,24 @@ public class Config extends HashMap<String, Object> {
 
 
     /**
-     * The maximum number of tuples that can be pending on a spout task at any given time.
-     * This config applies to individual tasks, not to spouts or topologies as a whole.
-     *
+     * The maximum number of tuples that can be pending on a spout task at any given time. 
+     * This config applies to individual tasks, not to spouts or topologies as a whole. 
+     * 
      * A pending tuple is one that has been emitted from a spout but has not been acked or failed yet.
-     * Note that this config parameter has no effect for unreliable spouts that don't tag
+     * Note that this config parameter has no effect for unreliable spouts that don't tag 
      * their tuples with a message id.
      */
-    public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending";
+    public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; 
     public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = Number.class;
 
     /**
      * A class that implements a strategy for what to do when a spout needs to wait. Waiting is
      * triggered in one of two conditions:
-     *
+     * 
      * 1. nextTuple emits no tuples
      * 2. The spout has hit maxSpoutPending and can't emit any more tuples
      */
-    public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy";
+    public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy"; 
     public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class;
 
     /**
@@ -625,7 +625,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA = Number.class;
 
     /**
-     * The time period that builtin metrics data in bucketed into.
+     * The time period that builtin metrics data in bucketed into. 
      */
     public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs";
     public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = Number.class;
@@ -652,7 +652,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A list of task hooks that are automatically added to every spout and bolt in the topology. An example
-     * of when you'd do this is to add a hook that integrates with your internal
+     * of when you'd do this is to add a hook that integrates with your internal 
      * monitoring system. These hooks are instantiated using the zero-arg constructor.
      */
     public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks";
@@ -666,7 +666,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator;
 
     /**
-     * The maximum number of messages to batch from the thread receiving off the network to the
+     * The maximum number of messages to batch from the thread receiving off the network to the 
      * executor queues. Must be a power of 2.
      */
     public static final String TOPOLOGY_RECEIVER_BUFFER_SIZE="topology.receiver.buffer.size";
@@ -700,14 +700,14 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class;
 
    /**
-    * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
+    * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed 
     * via the TopologyContext.
     */
     public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size";
     public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = Number.class;
 
     /**
-     * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
+     * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, 
      * an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
      * reported to Zookeeper per task for every 10 second interval of time.
      */
@@ -776,9 +776,9 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
-     * for the java.library.path value. java.library.path tells the JVM where
+     * for the java.library.path value. java.library.path tells the JVM where 
      * to look for native libraries. It is necessary to set this config correctly since
-     * Storm uses the ZeroMQ and JZMQ native libs.
+     * Storm uses the ZeroMQ and JZMQ native libs. 
      */
     public static final String JAVA_LIBRARY_PATH = "java.library.path";
     public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class;
@@ -800,17 +800,17 @@ public class Config extends HashMap<String, Object> {
 
     public static void setDebug(Map conf, boolean isOn) {
         conf.put(Config.TOPOLOGY_DEBUG, isOn);
-    }
+    } 
 
     public void setDebug(boolean isOn) {
         setDebug(this, isOn);
     }
-
+    
     @Deprecated
     public void setOptimize(boolean isOn) {
         put(Config.TOPOLOGY_OPTIMIZE, isOn);
-    }
-
+    } 
+    
     public static void setNumWorkers(Map conf, int workers) {
         conf.put(Config.TOPOLOGY_WORKERS, workers);
     }
@@ -826,7 +826,7 @@ public class Config extends HashMap<String, Object> {
     public void setNumAckers(int numExecutors) {
         setNumAckers(this, numExecutors);
     }
-
+    
     public static void setMessageTimeoutSecs(Map conf, int secs) {
         conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs);
     }
@@ -834,7 +834,7 @@ public class Config extends HashMap<String, Object> {
     public void setMessageTimeoutSecs(int secs) {
         setMessageTimeoutSecs(this, secs);
     }
-
+    
     public static void registerSerialization(Map conf, Class klass) {
         getRegisteredSerializations(conf).add(klass.getName());
     }
@@ -842,17 +842,17 @@ public class Config extends HashMap<String, Object> {
     public void registerSerialization(Class klass) {
         registerSerialization(this, klass);
     }
-
+    
     public static void registerSerialization(Map conf, Class klass, Class<? extends Serializer> serializerClass) {
         Map<String, String> register = new HashMap<String, String>();
         register.put(klass.getName(), serializerClass.getName());
-        getRegisteredSerializations(conf).add(register);
+        getRegisteredSerializations(conf).add(register);        
     }
 
     public void registerSerialization(Class klass, Class<? extends Serializer> serializerClass) {
         registerSerialization(this, klass, serializerClass);
     }
-
+    
     public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) {
         HashMap m = new HashMap();
         m.put("class", klass.getCanonicalName());
@@ -880,7 +880,7 @@ public class Config extends HashMap<String, Object> {
     public void registerDecorator(Class<? extends IKryoDecorator> klass) {
         registerDecorator(this, klass);
     }
-
+    
     public static void setKryoFactory(Map conf, Class<? extends IKryoFactory> klass) {
         conf.put(Config.TOPOLOGY_KRYO_FACTORY, klass.getName());
     }
@@ -896,7 +896,7 @@ public class Config extends HashMap<String, Object> {
     public void setSkipMissingKryoRegistrations(boolean skip) {
        setSkipMissingKryoRegistrations(this, skip);
     }
-
+    
     public static void setMaxTaskParallelism(Map conf, int max) {
         conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max);
     }
@@ -904,7 +904,7 @@ public class Config extends HashMap<String, Object> {
     public void setMaxTaskParallelism(int max) {
         setMaxTaskParallelism(this, max);
     }
-
+    
     public static void setMaxSpoutPending(Map conf, int max) {
         conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max);
     }
@@ -912,23 +912,23 @@ public class Config extends HashMap<String, Object> {
     public void setMaxSpoutPending(int max) {
         setMaxSpoutPending(this, max);
     }
-
+    
     public static void setStatsSampleRate(Map conf, double rate) {
         conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, rate);
-    }
+    }    
 
     public void setStatsSampleRate(double rate) {
         setStatsSampleRate(this, rate);
-    }
+    }    
 
     public static void setFallBackOnJavaSerialization(Map conf, boolean fallback) {
         conf.put(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION, fallback);
-    }
+    }    
 
     public void setFallBackOnJavaSerialization(boolean fallback) {
         setFallBackOnJavaSerialization(this, fallback);
-    }
-
+    }    
+    
     private static List getRegisteredSerializations(Map conf) {
         List ret;
         if(!conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) {
@@ -939,13 +939,13 @@ public class Config extends HashMap<String, Object> {
         conf.put(Config.TOPOLOGY_KRYO_REGISTER, ret);
         return ret;
     }
-
+    
     private static List getRegisteredDecorators(Map conf) {
         List ret;
         if(!conf.containsKey(Config.TOPOLOGY_KRYO_DECORATORS)) {
             ret = new ArrayList();
         } else {
-            ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS));
+            ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS));            
         }
         conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret);
         return ret;


[21/24] git commit: Converted ids to Objects to be compatible with py rb multilangs

Posted by bo...@apache.org.
Converted ids to Objects to be compatible with py rb multilangs


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

Branch: refs/heads/master
Commit: 0c669f429b6cf3fa1719dbb35b71b647cd241ab3
Parents: 660d8c4
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Thu May 1 14:26:31 2014 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Thu May 1 14:26:31 2014 +0200

----------------------------------------------------------------------
 .../src/jvm/backtype/storm/multilang/JsonSerializer.java       | 2 +-
 storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java      | 6 +++---
 storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java      | 6 +++---
 storm-core/src/jvm/backtype/storm/spout/ShellSpout.java        | 4 ++--
 storm-core/src/jvm/backtype/storm/task/ShellBolt.java          | 4 ++--
 5 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/0c669f42/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
index 70d8e5b..0d79659 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -81,7 +81,7 @@ public class JsonSerializer implements ISerializer {
         String command = (String) msg.get("command");
         shellMsg.setCommand(command);
 
-        String id = (String) msg.get("id");
+        Object id = msg.get("id");
         shellMsg.setId(id);
 
         String log = (String) msg.get("msg");

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/0c669f42/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
index 4381c07..1747f5b 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
@@ -17,7 +17,7 @@ import java.util.List;
  */
 public class ShellMsg {
     private String command;
-    private String id;
+    private Object id;
     private List<String> anchors;
     private String stream;
     private long task;
@@ -33,11 +33,11 @@ public class ShellMsg {
         this.command = command;
     }
 
-    public String getId() {
+    public Object getId() {
         return id;
     }
 
-    public void setId(String id) {
+    public void setId(Object id) {
         this.id = id;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/0c669f42/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
index 9b5ffce..d1ee030 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
@@ -14,7 +14,7 @@ package backtype.storm.multilang;
  */
 public class SpoutMsg {
     private String command;
-    private String id;
+    private Object id;
 
     public String getCommand() {
         return command;
@@ -24,11 +24,11 @@ public class SpoutMsg {
         this.command = command;
     }
 
-    public String getId() {
+    public Object getId() {
         return id;
     }
 
-    public void setId(String id) {
+    public void setId(Object id) {
         this.id = id;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/0c669f42/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index b28348c..87bc5a8 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -73,7 +73,7 @@ public class ShellSpout implements ISpout {
             spoutMsg = new SpoutMsg();
         }
         spoutMsg.setCommand("ack");
-        spoutMsg.setId(msgId.toString());
+        spoutMsg.setId(msgId);
         querySubprocess();
     }
 
@@ -82,7 +82,7 @@ public class ShellSpout implements ISpout {
             spoutMsg = new SpoutMsg();
         }
         spoutMsg.setCommand("fail");
-        spoutMsg.setId(msgId.toString());
+        spoutMsg.setId(msgId);
         querySubprocess();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/0c669f42/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index 27e55bb..81aca02 100644
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -180,7 +180,7 @@ public class ShellBolt implements IBolt {
         _inputs.clear();
     }
 
-    private void handleAck(String id) {
+    private void handleAck(Object id) {
         Tuple acked = _inputs.remove(id);
         if(acked==null) {
             throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id);
@@ -188,7 +188,7 @@ public class ShellBolt implements IBolt {
         _collector.ack(acked);
     }
 
-    private void handleFail(String id) {
+    private void handleFail(Object id) {
         Tuple failed = _inputs.remove(id);
         if(failed==null) {
             throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id);


[17/24] git commit: Added null check for anchors. Log a component's error stream under it name.

Posted by bo...@apache.org.
Added null check for anchors. Log a component's error stream under it name.


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

Branch: refs/heads/master
Commit: 8cde5508b5e4828982aa4f1ff2e70ba6fd10f4b9
Parents: 5ba0938
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Fri Nov 29 14:47:31 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Fri Nov 29 14:47:31 2013 +0200

----------------------------------------------------------------------
 .../jvm/backtype/storm/spout/ShellSpout.java    |  8 +++---
 .../src/jvm/backtype/storm/task/ShellBolt.java  | 19 +++++++------
 .../jvm/backtype/storm/utils/ShellProcess.java  | 28 +++++++++-----------
 3 files changed, 27 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/8cde5508/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index 6840938..bd5f84e 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -31,6 +31,7 @@ public class ShellSpout implements ISpout {
     public void open(Map stormConf, TopologyContext context,
                      SpoutOutputCollector collector) {
         _collector = collector;
+
         _process = new ShellProcess(_command);
 
         Number subpid = _process.launch(stormConf, context);
@@ -51,7 +52,7 @@ public class ShellSpout implements ISpout {
     }
 
     public void ack(Object msgId) {
-    	if (spoutMsg == null) {
+        if (spoutMsg == null) {
             spoutMsg = new SpoutMsg();
         }
         spoutMsg.setCommand("ack");
@@ -60,7 +61,7 @@ public class ShellSpout implements ISpout {
     }
 
     public void fail(Object msgId) {
-    	if (spoutMsg == null) {
+        if (spoutMsg == null) {
             spoutMsg = new SpoutMsg();
         }
         spoutMsg.setCommand("fail");
@@ -91,8 +92,7 @@ public class ShellSpout implements ISpout {
                             _process.writeTaskIds(outtasks);
                         }
                     } else {
-                        _collector.emitDirect((int) task.longValue(), stream,
-                                tuple, messageId);
+                        _collector.emitDirect((int) task.longValue(), stream, tuple, messageId);
                     }
                 } else {
                     throw new RuntimeException("Unknown command received: " + command);

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/8cde5508/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index 8efd904..0016d66 100644
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -184,14 +184,17 @@ public class ShellBolt implements IBolt {
     }
 
     private void handleEmit(ShellMsg shellMsg) throws InterruptedException {
-    	List<Tuple> anchors = new ArrayList<Tuple>();
-    	for (String anchor : shellMsg.getAnchors()) {
-	    	Tuple t = _inputs.get(anchor);
-	        if (t == null) {
-	            throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
-	        }
-	        anchors.add(t);
-    	}
+        List<Tuple> anchors = new ArrayList<Tuple>();
+        List<String> recvAnchors = shellMsg.getAnchors();
+        if (recvAnchors != null) {
+            for (String anchor : recvAnchors) {
+                Tuple t = _inputs.get(anchor);
+                if (t == null) {
+                    throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
+                }
+                anchors.add(t);
+            }
+        }
 
         if(shellMsg.getTask() == 0) {
             List<Integer> outtasks = _collector.emit(shellMsg.getStream(), anchors, shellMsg.getTuple());

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/8cde5508/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
index 8871a77..2c3b3a3 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
@@ -19,6 +19,7 @@ import org.apache.log4j.Logger;
 
 public class ShellProcess implements Serializable {
     public static Logger LOG = Logger.getLogger(ShellProcess.class);
+    public static Logger ShellLogger;
     private Process      _subprocess;
     private InputStream  processErrorStream;
     private String[]     command;
@@ -32,14 +33,15 @@ public class ShellProcess implements Serializable {
         ProcessBuilder builder = new ProcessBuilder(command);
         builder.directory(new File(context.getCodeDir()));
 
+        ShellLogger = Logger.getLogger(context.getThisComponentId());
+
         this.serializer = getSerializer(conf);
 
         Number pid;
         try {
             _subprocess = builder.start();
             processErrorStream = _subprocess.getErrorStream();
-            serializer.initialize(_subprocess.getOutputStream(),
-                    _subprocess.getInputStream());
+            serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream());
             pid = serializer.connect(conf, context);
         } catch (IOException e) {
             throw new RuntimeException(
@@ -83,35 +85,29 @@ public class ShellProcess implements Serializable {
 
     public void writeBoltMsg(BoltMsg msg) throws IOException {
         serializer.writeBoltMsg(msg);
-        // drain the error stream to avoid dead lock because of full error
-        // stream buffer
-        drainErrorStream();
+        // Log any info sent on the error stream
+        logErrorStream();
     }
 
     public void writeSpoutMsg(SpoutMsg msg) throws IOException {
         serializer.writeSpoutMsg(msg);
-        // drain the error stream to avoid dead lock because of full error
-        // stream buffer
-        drainErrorStream();
+        // Log any info sent on the error stream
+        logErrorStream();
     }
 
     public void writeTaskIds(List<Integer> taskIds) throws IOException {
         serializer.writeTaskIds(taskIds);
-        // drain the error stream to avoid dead lock because of full error
-        // stream buffer
-        drainErrorStream();
+        // Log any info sent on the error stream
+        logErrorStream();
     }
 
-    public void drainErrorStream() {
+    public void logErrorStream() {
         try {
             while (processErrorStream.available() > 0) {
                 int bufferSize = processErrorStream.available();
                 byte[] errorReadingBuffer = new byte[bufferSize];
-
                 processErrorStream.read(errorReadingBuffer, 0, bufferSize);
-
-                LOG.info("Got error from shell process: "
-                        + new String(errorReadingBuffer));
+                ShellLogger.info(new String(errorReadingBuffer));
             }
         } catch (Exception e) {
         }


[24/24] git commit: Added STORM-138 to Changelog

Posted by bo...@apache.org.
Added STORM-138 to Changelog


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

Branch: refs/heads/master
Commit: a09adda06fb4f9c2cf291ef9d72eba4cf9972e68
Parents: d3c6a1b
Author: Robert (Bobby) Evans <bo...@apache.org>
Authored: Mon May 19 21:39:47 2014 +0000
Committer: Robert (Bobby) Evans <bo...@apache.org>
Committed: Mon May 19 21:39:47 2014 +0000

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/a09adda0/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index a29dfb8..4a8f4ff 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.2-incubating (unreleased)
+ * STORM-138: Pluggable serialization for multilang
  * STORM-264: Removes references to the deprecated topology.optimize
  * STORM-245: implement Stream.localOrShuffle() for trident
  * STORM-317: Add SECURITY.md to release binaries


[16/24] git commit: Added debugging info to ShellSpout and pulled in latest upstream master.

Posted by bo...@apache.org.
Added debugging info to ShellSpout and pulled in latest upstream master.


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

Branch: refs/heads/master
Commit: 5ba093862560aa3ca2cff002e2e38e1e37753670
Parents: 8030423
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Mon Nov 18 16:31:44 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Mon Nov 18 16:31:44 2013 +0200

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/spout/ShellSpout.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/5ba09386/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index bee56fe..6840938 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -94,6 +94,8 @@ public class ShellSpout implements ISpout {
                         _collector.emitDirect((int) task.longValue(), stream,
                                 tuple, messageId);
                     }
+                } else {
+                    throw new RuntimeException("Unknown command received: " + command);
                 }
             }
         } catch (IOException e) {


[18/24] git commit: Merge branch 'master' of https://github.com/nathanmarz/storm

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/nathanmarz/storm

Conflicts:
	conf/defaults.yaml


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

Branch: refs/heads/master
Commit: 46c02be8181d196e48484807e34215d5e5fb61b4
Parents: 8cde550 b0f3310
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Wed Apr 23 09:12:01 2014 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Wed Apr 23 09:12:01 2014 +0200

----------------------------------------------------------------------
 .gitignore                                      |    4 +-
 CHANGELOG.md                                    |   21 +-
 DISCLAIMER                                      |   10 +
 KEYS                                            |   67 +-
 LICENSE                                         |  285 ++
 LICENSE.html                                    |  261 --
 MODULES                                         |    4 -
 NOTICE                                          |   12 +-
 README.markdown                                 |   23 +-
 VERSION                                         |    2 +-
 bin/build_modules.sh                            |   25 -
 bin/build_release.sh                            |   62 -
 bin/install_zmq.sh                              |   31 -
 bin/javadoc.sh                                  |    2 -
 bin/storm                                       |   18 +-
 bin/storm-config.cmd                            |  111 +
 bin/storm.cmd                                   |  201 ++
 bin/to_maven.sh                                 |   37 -
 conf/defaults.yaml                              |   23 +-
 conf/jaas_digest.conf                           |   18 +
 conf/logback.xml                                |   17 +-
 conf/storm.yaml.example                         |   16 +
 logback/cluster.xml                             |   18 +-
 pom.xml                                         |  649 +++++
 project.clj                                     |   16 -
 .../maven-shade-clojure-transformer/pom.xml     |   39 +
 .../maven/shade/clojure/ClojureTransformer.java |   72 +
 storm-console-logging/logback/logback.xml       |    1 -
 storm-console-logging/project.clj               |   11 -
 storm-core/pom.xml                              |  266 ++
 storm-core/project.clj                          |   52 -
 .../src/clj/backtype/storm/LocalCluster.clj     |   15 +
 storm-core/src/clj/backtype/storm/LocalDRPC.clj |   15 +
 storm-core/src/clj/backtype/storm/bootstrap.clj |   15 +
 storm-core/src/clj/backtype/storm/clojure.clj   |   15 +
 storm-core/src/clj/backtype/storm/cluster.clj   |   15 +
 .../src/clj/backtype/storm/command/activate.clj |   15 +
 .../clj/backtype/storm/command/config_value.clj |   15 +
 .../clj/backtype/storm/command/deactivate.clj   |   15 +
 .../backtype/storm/command/dev_zookeeper.clj    |   15 +
 .../backtype/storm/command/kill_topology.clj    |   15 +
 .../src/clj/backtype/storm/command/list.clj     |   15 +
 .../clj/backtype/storm/command/rebalance.clj    |   15 +
 .../backtype/storm/command/shell_submission.clj |   15 +
 storm-core/src/clj/backtype/storm/config.clj    |   63 +-
 .../src/clj/backtype/storm/daemon/acker.clj     |   15 +
 .../backtype/storm/daemon/builtin_metrics.clj   |   15 +
 .../src/clj/backtype/storm/daemon/common.clj    |   15 +
 .../src/clj/backtype/storm/daemon/drpc.clj      |   27 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |   15 +
 .../src/clj/backtype/storm/daemon/logviewer.clj |   17 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   42 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |   52 +-
 .../src/clj/backtype/storm/daemon/task.clj      |   15 +
 .../src/clj/backtype/storm/daemon/worker.clj    |   15 +
 storm-core/src/clj/backtype/storm/disruptor.clj |   15 +
 storm-core/src/clj/backtype/storm/event.clj     |   15 +
 storm-core/src/clj/backtype/storm/log.clj       |   15 +
 .../src/clj/backtype/storm/messaging/loader.clj |   15 +
 .../src/clj/backtype/storm/messaging/local.clj  |   15 +
 .../src/clj/backtype/storm/messaging/zmq.clj    |   93 -
 .../src/clj/backtype/storm/metric/testing.clj   |   15 +
 .../clj/backtype/storm/process_simulator.clj    |   15 +
 .../storm/scheduler/DefaultScheduler.clj        |   15 +
 .../backtype/storm/scheduler/EvenScheduler.clj  |   15 +
 .../storm/scheduler/IsolationScheduler.clj      |   15 +
 storm-core/src/clj/backtype/storm/stats.clj     |   15 +
 storm-core/src/clj/backtype/storm/testing.clj   |   23 +-
 storm-core/src/clj/backtype/storm/testing4j.clj |   16 +
 storm-core/src/clj/backtype/storm/thrift.clj    |   19 +-
 storm-core/src/clj/backtype/storm/timer.clj     |   15 +
 storm-core/src/clj/backtype/storm/tuple.clj     |   15 +
 storm-core/src/clj/backtype/storm/ui/core.clj   |  327 ++-
 .../src/clj/backtype/storm/ui/helpers.clj       |   21 +-
 storm-core/src/clj/backtype/storm/util.clj      |   45 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj |   15 +
 storm-core/src/clj/storm/trident/testing.clj    |   16 +
 storm-core/src/clj/zilch/mq.clj                 |  104 -
 storm-core/src/dev/resources/tester_bolt.py     |   18 +
 storm-core/src/dev/resources/tester_bolt.rb     |   18 +
 storm-core/src/dev/resources/tester_spout.py    |   18 +
 storm-core/src/dev/resources/tester_spout.rb    |   17 +
 storm-core/src/genthrift.sh                     |   16 +
 storm-core/src/jvm/backtype/storm/Config.java   |   25 +-
 .../jvm/backtype/storm/ConfigValidation.java    |   49 +
 .../src/jvm/backtype/storm/Constants.java       |   17 +
 .../src/jvm/backtype/storm/ILocalCluster.java   |   17 +
 .../src/jvm/backtype/storm/ILocalDRPC.java      |   17 +
 .../src/jvm/backtype/storm/StormSubmitter.java  |   21 +-
 .../jvm/backtype/storm/clojure/ClojureBolt.java |   17 +
 .../backtype/storm/clojure/ClojureSpout.java    |   17 +
 .../backtype/storm/clojure/RichShellBolt.java   |   17 +
 .../backtype/storm/clojure/RichShellSpout.java  |   17 +
 .../storm/coordination/BatchBoltExecutor.java   |   17 +
 .../coordination/BatchOutputCollector.java      |   17 +
 .../coordination/BatchOutputCollectorImpl.java  |   17 +
 .../coordination/BatchSubtopologyBuilder.java   |   17 +
 .../storm/coordination/CoordinatedBolt.java     |   17 +
 .../backtype/storm/coordination/IBatchBolt.java |   17 +
 .../jvm/backtype/storm/daemon/Shutdownable.java |   17 +
 .../storm/drpc/DRPCInvocationsClient.java       |   27 +-
 .../src/jvm/backtype/storm/drpc/DRPCSpout.java  |   21 +-
 .../src/jvm/backtype/storm/drpc/JoinResult.java |   17 +
 .../jvm/backtype/storm/drpc/KeyedFairBolt.java  |   17 +
 .../storm/drpc/LinearDRPCInputDeclarer.java     |   17 +
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |   17 +
 .../jvm/backtype/storm/drpc/PrepareRequest.java |   17 +
 .../jvm/backtype/storm/drpc/ReturnResults.java  |   19 +-
 .../storm/generated/AlreadyAliveException.java  |   63 +-
 .../src/jvm/backtype/storm/generated/Bolt.java  |   77 +-
 .../jvm/backtype/storm/generated/BoltStats.java |  199 +-
 .../storm/generated/ClusterSummary.java         |  103 +-
 .../storm/generated/ComponentCommon.java        |  117 +-
 .../storm/generated/ComponentObject.java        |   83 +-
 .../storm/generated/DRPCExecutionException.java |   63 +-
 .../backtype/storm/generated/DRPCRequest.java   |   77 +-
 .../storm/generated/DistributedRPC.java         |  197 +-
 .../generated/DistributedRPCInvocations.java    |  401 +--
 .../jvm/backtype/storm/generated/ErrorInfo.java |   77 +-
 .../backtype/storm/generated/ExecutorInfo.java  |   77 +-
 .../storm/generated/ExecutorSpecificStats.java  |   73 +-
 .../backtype/storm/generated/ExecutorStats.java |  123 +-
 .../storm/generated/ExecutorSummary.java        |  131 +-
 .../storm/generated/GlobalStreamId.java         |   77 +-
 .../jvm/backtype/storm/generated/Grouping.java  |  129 +-
 .../generated/InvalidTopologyException.java     |   63 +-
 .../backtype/storm/generated/JavaObject.java    |   83 +-
 .../backtype/storm/generated/JavaObjectArg.java |  107 +-
 .../backtype/storm/generated/KillOptions.java   |   61 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 2385 ++++++++---------
 .../storm/generated/NotAliveException.java      |   63 +-
 .../backtype/storm/generated/NullStruct.java    |   49 +-
 .../storm/generated/RebalanceOptions.java       |   93 +-
 .../storm/generated/ShellComponent.java         |   73 +-
 .../jvm/backtype/storm/generated/SpoutSpec.java |   77 +-
 .../backtype/storm/generated/SpoutStats.java    |  139 +-
 .../storm/generated/StateSpoutSpec.java         |   77 +-
 .../backtype/storm/generated/StormTopology.java |  115 +-
 .../backtype/storm/generated/StreamInfo.java    |   83 +-
 .../backtype/storm/generated/SubmitOptions.java |   63 +-
 .../storm/generated/SupervisorSummary.java      |  119 +-
 .../backtype/storm/generated/TopologyInfo.java  |  153 +-
 .../storm/generated/TopologyInitialStatus.java  |   21 +-
 .../storm/generated/TopologySummary.java        |  147 +-
 .../storm/grouping/CustomStreamGrouping.java    |   17 +
 .../jvm/backtype/storm/hooks/BaseTaskHook.java  |   17 +
 .../src/jvm/backtype/storm/hooks/ITaskHook.java |   17 +
 .../backtype/storm/hooks/info/BoltAckInfo.java  |   17 +
 .../storm/hooks/info/BoltExecuteInfo.java       |   17 +
 .../backtype/storm/hooks/info/BoltFailInfo.java |   17 +
 .../jvm/backtype/storm/hooks/info/EmitInfo.java |   17 +
 .../backtype/storm/hooks/info/SpoutAckInfo.java |   17 +
 .../storm/hooks/info/SpoutFailInfo.java         |   17 +
 .../backtype/storm/messaging/IConnection.java   |   17 +
 .../jvm/backtype/storm/messaging/IContext.java  |   17 +
 .../backtype/storm/messaging/TaskMessage.java   |   17 +
 .../storm/messaging/TransportFactory.java       |   17 +
 .../backtype/storm/messaging/netty/Client.java  |  221 ++
 .../backtype/storm/messaging/netty/Context.java |   67 +
 .../storm/messaging/netty/ControlMessage.java   |   67 +
 .../storm/messaging/netty/MessageBatch.java     |  168 ++
 .../storm/messaging/netty/MessageDecoder.java   |   85 +
 .../storm/messaging/netty/MessageEncoder.java   |   39 +
 .../backtype/storm/messaging/netty/Server.java  |  136 +
 .../messaging/netty/StormClientHandler.java     |  121 +
 .../netty/StormClientPipelineFactory.java       |   44 +
 .../messaging/netty/StormServerHandler.java     |   70 +
 .../netty/StormServerPipelineFactory.java       |   45 +
 .../storm/metric/LoggingMetricsConsumer.java    |   17 +
 .../storm/metric/MetricsConsumerBolt.java       |   17 +
 .../jvm/backtype/storm/metric/SystemBolt.java   |   17 +
 .../storm/metric/api/AssignableMetric.java      |   17 +
 .../storm/metric/api/CombinedMetric.java        |   17 +
 .../backtype/storm/metric/api/CountMetric.java  |   17 +
 .../backtype/storm/metric/api/ICombiner.java    |   17 +
 .../jvm/backtype/storm/metric/api/IMetric.java  |   17 +
 .../storm/metric/api/IMetricsConsumer.java      |   17 +
 .../jvm/backtype/storm/metric/api/IReducer.java |   17 +
 .../storm/metric/api/IStatefulObject.java       |   17 +
 .../backtype/storm/metric/api/MeanReducer.java  |   17 +
 .../storm/metric/api/MultiCountMetric.java      |   17 +
 .../storm/metric/api/MultiReducedMetric.java    |   17 +
 .../storm/metric/api/ReducedMetric.java         |   17 +
 .../backtype/storm/metric/api/StateMetric.java  |   17 +
 .../storm/nimbus/DefaultTopologyValidator.java  |   17 +
 .../storm/nimbus/ITopologyValidator.java        |   17 +
 .../backtype/storm/planner/CompoundSpout.java   |   17 +
 .../backtype/storm/planner/CompoundTask.java    |   17 +
 .../jvm/backtype/storm/planner/TaskBundle.java  |   17 +
 .../jvm/backtype/storm/scheduler/Cluster.java   |   17 +
 .../storm/scheduler/ExecutorDetails.java        |   91 +-
 .../jvm/backtype/storm/scheduler/INimbus.java   |   17 +
 .../backtype/storm/scheduler/IScheduler.java    |   17 +
 .../backtype/storm/scheduler/ISupervisor.java   |   17 +
 .../storm/scheduler/SchedulerAssignment.java    |   97 +-
 .../scheduler/SchedulerAssignmentImpl.java      |  201 +-
 .../storm/scheduler/SupervisorDetails.java      |   17 +
 .../backtype/storm/scheduler/Topologies.java    |   97 +-
 .../storm/scheduler/TopologyDetails.java        |   17 +
 .../backtype/storm/scheduler/WorkerSlot.java    |   17 +
 .../backtype/storm/security/auth/AuthUtils.java |   17 +
 .../storm/security/auth/IAuthorizer.java        |   17 +
 .../storm/security/auth/ITransportPlugin.java   |   25 +-
 .../storm/security/auth/ReqContext.java         |   17 +
 .../security/auth/SaslTransportPlugin.java      |   41 +-
 .../security/auth/SimpleTransportPlugin.java    |   41 +-
 .../storm/security/auth/ThriftClient.java       |   27 +-
 .../storm/security/auth/ThriftServer.java       |   21 +-
 .../auth/authorizer/DenyAuthorizer.java         |   17 +
 .../auth/authorizer/NoopAuthorizer.java         |   17 +
 .../auth/digest/ClientCallbackHandler.java      |   17 +
 .../auth/digest/DigestSaslTransportPlugin.java  |   27 +-
 .../auth/digest/ServerCallbackHandler.java      |   17 +
 .../serialization/BlowfishTupleSerializer.java  |   17 +
 .../storm/serialization/DefaultKryoFactory.java |   17 +
 .../storm/serialization/IKryoDecorator.java     |   17 +
 .../storm/serialization/IKryoFactory.java       |   17 +
 .../storm/serialization/ITupleDeserializer.java |   17 +
 .../storm/serialization/ITupleSerializer.java   |   17 +
 .../serialization/KryoTupleDeserializer.java    |   17 +
 .../serialization/KryoTupleSerializer.java      |   17 +
 .../serialization/KryoValuesDeserializer.java   |   17 +
 .../serialization/KryoValuesSerializer.java     |   17 +
 .../serialization/SerializableSerializer.java   |   17 +
 .../serialization/SerializationFactory.java     |   17 +
 .../types/ArrayListSerializer.java              |   17 +
 .../serialization/types/HashMapSerializer.java  |   17 +
 .../serialization/types/HashSetSerializer.java  |   17 +
 .../types/ListDelegateSerializer.java           |   17 +
 .../storm/spout/IMultiSchemableSpout.java       |   17 +
 .../backtype/storm/spout/ISchemableSpout.java   |   17 +
 .../src/jvm/backtype/storm/spout/ISpout.java    |   17 +
 .../storm/spout/ISpoutOutputCollector.java      |   17 +
 .../storm/spout/ISpoutWaitStrategy.java         |   17 +
 .../jvm/backtype/storm/spout/MultiScheme.java   |   17 +
 .../storm/spout/NothingEmptyEmitStrategy.java   |   17 +
 .../backtype/storm/spout/RawMultiScheme.java    |   17 +
 .../src/jvm/backtype/storm/spout/RawScheme.java |   17 +
 .../src/jvm/backtype/storm/spout/Scheme.java    |   17 +
 .../storm/spout/SchemeAsMultiScheme.java        |   17 +
 .../jvm/backtype/storm/spout/ShellSpout.java    |   17 +
 .../storm/spout/SleepSpoutWaitStrategy.java     |   17 +
 .../storm/spout/SpoutOutputCollector.java       |   17 +
 .../jvm/backtype/storm/state/IStateSpout.java   |   17 +
 .../storm/state/IStateSpoutOutputCollector.java |   17 +
 .../backtype/storm/state/ISubscribedState.java  |   17 +
 .../state/ISynchronizeOutputCollector.java      |   17 +
 .../storm/state/StateSpoutOutputCollector.java  |   17 +
 .../storm/state/SynchronizeOutputCollector.java |   17 +
 .../storm/task/GeneralTopologyContext.java      |   17 +
 .../src/jvm/backtype/storm/task/IBolt.java      |   17 +
 .../jvm/backtype/storm/task/IErrorReporter.java |   17 +
 .../backtype/storm/task/IMetricsContext.java    |   17 +
 .../backtype/storm/task/IOutputCollector.java   |   17 +
 .../backtype/storm/task/OutputCollector.java    |   17 +
 .../src/jvm/backtype/storm/task/ShellBolt.java  |   17 +
 .../backtype/storm/task/TopologyContext.java    |   17 +
 .../storm/task/WorkerTopologyContext.java       |   17 +
 .../backtype/storm/testing/AckFailDelegate.java |   17 +
 .../storm/testing/AckFailMapTracker.java        |   17 +
 .../jvm/backtype/storm/testing/AckTracker.java  |   17 +
 .../backtype/storm/testing/BatchNumberList.java |   17 +
 .../storm/testing/BatchProcessWord.java         |   17 +
 .../backtype/storm/testing/BatchRepeatA.java    |   17 +
 .../jvm/backtype/storm/testing/BoltTracker.java |   17 +
 .../storm/testing/CompleteTopologyParam.java    |  117 +-
 .../storm/testing/CountingBatchBolt.java        |   17 +
 .../storm/testing/CountingCommitBolt.java       |   17 +
 .../jvm/backtype/storm/testing/FeederSpout.java |   17 +
 .../jvm/backtype/storm/testing/FixedTuple.java  |   17 +
 .../backtype/storm/testing/FixedTupleSpout.java |   17 +
 .../backtype/storm/testing/IdentityBolt.java    |   17 +
 .../storm/testing/KeyedCountingBatchBolt.java   |   17 +
 .../testing/KeyedCountingCommitterBolt.java     |   17 +
 .../storm/testing/KeyedSummingBatchBolt.java    |   17 +
 .../storm/testing/MemoryTransactionalSpout.java |   17 +
 .../testing/MemoryTransactionalSpoutMeta.java   |   17 +
 .../backtype/storm/testing/MkClusterParam.java  |   97 +-
 .../backtype/storm/testing/MkTupleParam.java    |   85 +-
 .../backtype/storm/testing/MockedSources.java   |  103 +-
 .../jvm/backtype/storm/testing/NGrouping.java   |   17 +
 .../storm/testing/NonRichBoltTracker.java       |   17 +
 .../testing/OpaqueMemoryTransactionalSpout.java |   17 +
 .../storm/testing/PrepareBatchBolt.java         |   17 +
 .../backtype/storm/testing/SpoutTracker.java    |   17 +
 .../storm/testing/TestAggregatesCounter.java    |   17 +
 .../backtype/storm/testing/TestConfBolt.java    |   17 +
 .../backtype/storm/testing/TestGlobalCount.java |   17 +
 .../src/jvm/backtype/storm/testing/TestJob.java |   65 +-
 .../storm/testing/TestKryoDecorator.java        |   17 +
 .../backtype/storm/testing/TestPlannerBolt.java |   17 +
 .../storm/testing/TestPlannerSpout.java         |   17 +
 .../backtype/storm/testing/TestSerObject.java   |   17 +
 .../backtype/storm/testing/TestWordCounter.java |   17 +
 .../backtype/storm/testing/TestWordSpout.java   |   17 +
 .../backtype/storm/testing/TrackedTopology.java |   51 +-
 .../storm/testing/TupleCaptureBolt.java         |   17 +
 .../topology/BaseConfigurationDeclarer.java     |   17 +
 .../storm/topology/BasicBoltExecutor.java       |   17 +
 .../storm/topology/BasicOutputCollector.java    |   17 +
 .../backtype/storm/topology/BoltDeclarer.java   |   17 +
 .../ComponentConfigurationDeclarer.java         |   17 +
 .../storm/topology/FailedException.java         |   17 +
 .../jvm/backtype/storm/topology/IBasicBolt.java |   17 +
 .../storm/topology/IBasicOutputCollector.java   |   17 +
 .../jvm/backtype/storm/topology/IComponent.java |   17 +
 .../jvm/backtype/storm/topology/IRichBolt.java  |   17 +
 .../jvm/backtype/storm/topology/IRichSpout.java |   17 +
 .../storm/topology/IRichStateSpout.java         |   17 +
 .../backtype/storm/topology/InputDeclarer.java  |   17 +
 .../storm/topology/OutputFieldsDeclarer.java    |   17 +
 .../storm/topology/OutputFieldsGetter.java      |   17 +
 .../storm/topology/ReportedFailedException.java |   17 +
 .../backtype/storm/topology/SpoutDeclarer.java  |   17 +
 .../storm/topology/TopologyBuilder.java         |   17 +
 .../storm/topology/base/BaseBasicBolt.java      |   17 +
 .../storm/topology/base/BaseBatchBolt.java      |   17 +
 .../storm/topology/base/BaseComponent.java      |   17 +
 ...BaseOpaquePartitionedTransactionalSpout.java |   17 +
 .../base/BasePartitionedTransactionalSpout.java |   17 +
 .../storm/topology/base/BaseRichBolt.java       |   17 +
 .../storm/topology/base/BaseRichSpout.java      |   17 +
 .../topology/base/BaseTransactionalBolt.java    |   17 +
 .../topology/base/BaseTransactionalSpout.java   |   17 +
 .../storm/transactional/ICommitter.java         |   17 +
 .../ICommitterTransactionalSpout.java           |   17 +
 .../transactional/ITransactionalSpout.java      |   17 +
 .../storm/transactional/TransactionAttempt.java |   17 +
 .../TransactionalSpoutBatchExecutor.java        |   17 +
 .../TransactionalSpoutCoordinator.java          |   17 +
 .../TransactionalTopologyBuilder.java           |   17 +
 .../IOpaquePartitionedTransactionalSpout.java   |   17 +
 .../IPartitionedTransactionalSpout.java         |   17 +
 ...uePartitionedTransactionalSpoutExecutor.java |   17 +
 .../PartitionedTransactionalSpoutExecutor.java  |   17 +
 .../state/RotatingTransactionalState.java       |   17 +
 .../transactional/state/TransactionalState.java |   17 +
 .../src/jvm/backtype/storm/tuple/Fields.java    |   17 +
 .../src/jvm/backtype/storm/tuple/MessageId.java |   17 +
 .../src/jvm/backtype/storm/tuple/Tuple.java     |   17 +
 .../src/jvm/backtype/storm/tuple/TupleImpl.java |   17 +
 .../src/jvm/backtype/storm/tuple/Values.java    |   17 +
 .../storm/utils/BufferFileInputStream.java      |   17 +
 .../backtype/storm/utils/CRC32OutputStream.java |   17 +
 .../backtype/storm/utils/ClojureTimerTask.java  |   17 +
 .../src/jvm/backtype/storm/utils/Container.java |   17 +
 .../jvm/backtype/storm/utils/DRPCClient.java    |   27 +-
 .../backtype/storm/utils/DisruptorQueue.java    |   17 +
 .../storm/utils/IndifferentAccessMap.java       |   17 +
 .../backtype/storm/utils/InprocMessaging.java   |   17 +
 .../storm/utils/KeyedRoundRobinQueue.java       |   17 +
 .../jvm/backtype/storm/utils/ListDelegate.java  |   17 +
 .../jvm/backtype/storm/utils/LocalState.java    |   17 +
 .../jvm/backtype/storm/utils/MutableInt.java    |   17 +
 .../jvm/backtype/storm/utils/MutableLong.java   |   17 +
 .../jvm/backtype/storm/utils/MutableObject.java |   17 +
 .../jvm/backtype/storm/utils/NimbusClient.java  |   19 +-
 .../storm/utils/RegisteredGlobalState.java      |   17 +
 .../jvm/backtype/storm/utils/RotatingMap.java   |   17 +
 .../backtype/storm/utils/ServiceRegistry.java   |   17 +
 .../jvm/backtype/storm/utils/ShellProcess.java  |   17 +
 .../storm/utils/ThriftTopologyUtils.java        |   17 +
 .../src/jvm/backtype/storm/utils/Time.java      |   17 +
 .../jvm/backtype/storm/utils/TimeCacheMap.java  |   17 +
 .../src/jvm/backtype/storm/utils/Utils.java     |   19 +-
 .../backtype/storm/utils/VersionedStore.java    |   17 +
 .../storm/utils/WindowedTimeThrottler.java      |   17 +
 .../jvm/backtype/storm/utils/WritableUtils.java |   17 +
 .../backtype/storm/utils/ZookeeperAuthInfo.java |   17 +
 storm-core/src/jvm/storm/trident/JoinType.java  |   17 +
 storm-core/src/jvm/storm/trident/Stream.java    |   17 +
 .../src/jvm/storm/trident/TridentState.java     |   17 +
 .../src/jvm/storm/trident/TridentTopology.java  |   17 +
 .../trident/drpc/ReturnResultsReducer.java      |   19 +-
 .../fluent/ChainedAggregatorDeclarer.java       |   17 +
 .../fluent/ChainedFullAggregatorDeclarer.java   |   17 +
 .../ChainedPartitionAggregatorDeclarer.java     |   17 +
 .../trident/fluent/GlobalAggregationScheme.java |   17 +
 .../jvm/storm/trident/fluent/GroupedStream.java |   17 +
 .../trident/fluent/IAggregatableStream.java     |   17 +
 .../fluent/IChainedAggregatorDeclarer.java      |   17 +
 .../jvm/storm/trident/fluent/UniqueIdGen.java   |   17 +
 .../jvm/storm/trident/graph/GraphGrouper.java   |   17 +
 .../src/jvm/storm/trident/graph/Group.java      |   17 +
 .../jvm/storm/trident/operation/Aggregator.java |   17 +
 .../jvm/storm/trident/operation/Assembly.java   |   17 +
 .../storm/trident/operation/BaseAggregator.java |   17 +
 .../jvm/storm/trident/operation/BaseFilter.java |   17 +
 .../storm/trident/operation/BaseFunction.java   |   17 +
 .../trident/operation/BaseMultiReducer.java     |   17 +
 .../storm/trident/operation/BaseOperation.java  |   17 +
 .../trident/operation/CombinerAggregator.java   |   17 +
 .../storm/trident/operation/EachOperation.java  |   17 +
 .../src/jvm/storm/trident/operation/Filter.java |   17 +
 .../jvm/storm/trident/operation/Function.java   |   17 +
 .../trident/operation/GroupedMultiReducer.java  |   17 +
 .../storm/trident/operation/MultiReducer.java   |   17 +
 .../jvm/storm/trident/operation/Operation.java  |   17 +
 .../trident/operation/ReducerAggregator.java    |   17 +
 .../trident/operation/TridentCollector.java     |   17 +
 .../operation/TridentMultiReducerContext.java   |   17 +
 .../operation/TridentOperationContext.java      |   17 +
 .../storm/trident/operation/builtin/Count.java  |   17 +
 .../storm/trident/operation/builtin/Debug.java  |   17 +
 .../storm/trident/operation/builtin/Equals.java |   17 +
 .../trident/operation/builtin/FilterNull.java   |   17 +
 .../storm/trident/operation/builtin/FirstN.java |   17 +
 .../storm/trident/operation/builtin/MapGet.java |   17 +
 .../storm/trident/operation/builtin/Negate.java |   17 +
 .../trident/operation/builtin/SnapshotGet.java  |   17 +
 .../storm/trident/operation/builtin/Sum.java    |   17 +
 .../operation/builtin/TupleCollectionGet.java   |   17 +
 .../operation/impl/CaptureCollector.java        |   17 +
 .../operation/impl/ChainedAggregatorImpl.java   |   17 +
 .../trident/operation/impl/ChainedResult.java   |   17 +
 .../operation/impl/CombinerAggStateUpdater.java |   17 +
 .../impl/CombinerAggregatorCombineImpl.java     |   17 +
 .../impl/CombinerAggregatorInitImpl.java        |   17 +
 .../trident/operation/impl/FilterExecutor.java  |   17 +
 .../operation/impl/GlobalBatchToPartition.java  |   17 +
 .../trident/operation/impl/GroupCollector.java  |   17 +
 .../operation/impl/GroupedAggregator.java       |   17 +
 .../impl/GroupedMultiReducerExecutor.java       |   17 +
 .../operation/impl/IdentityMultiReducer.java    |   17 +
 .../impl/IndexHashBatchToPartition.java         |   17 +
 .../operation/impl/JoinerMultiReducer.java      |   17 +
 .../operation/impl/ReducerAggStateUpdater.java  |   17 +
 .../operation/impl/ReducerAggregatorImpl.java   |   17 +
 .../storm/trident/operation/impl/Result.java    |   17 +
 .../operation/impl/SingleEmitAggregator.java    |   17 +
 .../trident/operation/impl/TrueFilter.java      |   17 +
 .../storm/trident/partition/GlobalGrouping.java |   17 +
 .../trident/partition/IdentityGrouping.java     |   17 +
 .../trident/partition/IndexHashGrouping.java    |   17 +
 .../storm/trident/planner/BridgeReceiver.java   |   17 +
 .../src/jvm/storm/trident/planner/Node.java     |   17 +
 .../storm/trident/planner/NodeStateInfo.java    |   17 +
 .../storm/trident/planner/PartitionNode.java    |   17 +
 .../storm/trident/planner/ProcessorContext.java |   17 +
 .../storm/trident/planner/ProcessorNode.java    |   17 +
 .../jvm/storm/trident/planner/SpoutNode.java    |   17 +
 .../storm/trident/planner/SubtopologyBolt.java  |   17 +
 .../storm/trident/planner/TridentProcessor.java |   17 +
 .../storm/trident/planner/TupleReceiver.java    |   17 +
 .../planner/processor/AggregateProcessor.java   |   17 +
 .../planner/processor/AppendCollector.java      |   17 +
 .../planner/processor/EachProcessor.java        |   17 +
 .../planner/processor/FreshCollector.java       |   17 +
 .../processor/MultiReducerProcessor.java        |   17 +
 .../processor/PartitionPersistProcessor.java    |   17 +
 .../planner/processor/ProjectedProcessor.java   |   17 +
 .../planner/processor/StateQueryProcessor.java  |   17 +
 .../planner/processor/TridentContext.java       |   17 +
 .../storm/trident/spout/BatchSpoutExecutor.java |   17 +
 .../src/jvm/storm/trident/spout/IBatchID.java   |   17 +
 .../jvm/storm/trident/spout/IBatchSpout.java    |   17 +
 .../trident/spout/ICommitterTridentSpout.java   |   17 +
 .../spout/IOpaquePartitionedTridentSpout.java   |   17 +
 .../trident/spout/IPartitionedTridentSpout.java |   17 +
 .../storm/trident/spout/ISpoutPartition.java    |   17 +
 .../jvm/storm/trident/spout/ITridentSpout.java  |   17 +
 .../OpaquePartitionedTridentSpoutExecutor.java  |   17 +
 .../spout/PartitionedTridentSpoutExecutor.java  |   17 +
 .../trident/spout/RichSpoutBatchExecutor.java   |   17 +
 .../storm/trident/spout/RichSpoutBatchId.java   |   17 +
 .../spout/RichSpoutBatchIdSerializer.java       |   17 +
 .../trident/spout/RichSpoutBatchTriggerer.java  |   17 +
 .../trident/spout/TridentSpoutCoordinator.java  |   17 +
 .../trident/spout/TridentSpoutExecutor.java     |   17 +
 .../storm/trident/state/BaseQueryFunction.java  |   17 +
 .../storm/trident/state/BaseStateUpdater.java   |   17 +
 .../trident/state/CombinerValueUpdater.java     |   17 +
 .../storm/trident/state/ITupleCollection.java   |   17 +
 .../state/JSONNonTransactionalSerializer.java   |   17 +
 .../trident/state/JSONOpaqueSerializer.java     |   17 +
 .../state/JSONTransactionalSerializer.java      |   17 +
 .../jvm/storm/trident/state/OpaqueValue.java    |   17 +
 .../jvm/storm/trident/state/QueryFunction.java  |   17 +
 .../jvm/storm/trident/state/ReadOnlyState.java  |   17 +
 .../trident/state/ReducerValueUpdater.java      |   17 +
 .../src/jvm/storm/trident/state/Serializer.java |   17 +
 .../src/jvm/storm/trident/state/State.java      |   17 +
 .../jvm/storm/trident/state/StateFactory.java   |   17 +
 .../src/jvm/storm/trident/state/StateSpec.java  |   17 +
 .../src/jvm/storm/trident/state/StateType.java  |   17 +
 .../jvm/storm/trident/state/StateUpdater.java   |   17 +
 .../storm/trident/state/TransactionalValue.java |   17 +
 .../jvm/storm/trident/state/ValueUpdater.java   |   17 +
 .../trident/state/map/CachedBatchReadsMap.java  |   17 +
 .../jvm/storm/trident/state/map/CachedMap.java  |   17 +
 .../storm/trident/state/map/IBackingMap.java    |   17 +
 .../state/map/MapCombinerAggStateUpdater.java   |   17 +
 .../state/map/MapReducerAggStateUpdater.java    |   17 +
 .../jvm/storm/trident/state/map/MapState.java   |   17 +
 .../state/map/MicroBatchIBackingMap.java        |   17 +
 .../trident/state/map/NonTransactionalMap.java  |   17 +
 .../jvm/storm/trident/state/map/OpaqueMap.java  |   17 +
 .../trident/state/map/ReadOnlyMapState.java     |   17 +
 .../trident/state/map/SnapshottableMap.java     |   17 +
 .../trident/state/map/TransactionalMap.java     |   17 +
 .../state/snapshot/ReadOnlySnapshottable.java   |   17 +
 .../trident/state/snapshot/Snapshottable.java   |   17 +
 .../trident/testing/CountAsAggregator.java      |   17 +
 .../storm/trident/testing/FeederBatchSpout.java |   17 +
 .../testing/FeederCommitterBatchSpout.java      |   17 +
 .../storm/trident/testing/FixedBatchSpout.java  |   17 +
 .../src/jvm/storm/trident/testing/IFeeder.java  |   17 +
 .../trident/testing/LRUMemoryMapState.java      |   17 +
 .../storm/trident/testing/MemoryBackingMap.java |   17 +
 .../storm/trident/testing/MemoryMapState.java   |   17 +
 .../storm/trident/testing/MockTridentTuple.java |   17 +
 .../src/jvm/storm/trident/testing/Split.java    |   17 +
 .../jvm/storm/trident/testing/StringLength.java |   17 +
 .../jvm/storm/trident/testing/TrueFilter.java   |   17 +
 .../jvm/storm/trident/testing/TuplifyArgs.java  |   17 +
 .../jvm/storm/trident/topology/BatchInfo.java   |   17 +
 .../trident/topology/ITridentBatchBolt.java     |   17 +
 .../topology/MasterBatchCoordinator.java        |   17 +
 .../trident/topology/TransactionAttempt.java    |   17 +
 .../trident/topology/TridentBoltExecutor.java   |   17 +
 .../topology/TridentTopologyBuilder.java        |   17 +
 .../state/RotatingTransactionalState.java       |   17 +
 .../topology/state/TransactionalState.java      |   17 +
 .../src/jvm/storm/trident/tuple/ComboList.java  |   17 +
 .../src/jvm/storm/trident/tuple/ConsList.java   |   17 +
 .../jvm/storm/trident/tuple/TridentTuple.java   |   17 +
 .../storm/trident/tuple/TridentTupleView.java   |   17 +
 .../jvm/storm/trident/tuple/ValuePointer.java   |   17 +
 .../storm/trident/util/ErrorEdgeFactory.java    |   17 +
 .../src/jvm/storm/trident/util/IndexedEdge.java |   17 +
 .../src/jvm/storm/trident/util/LRUMap.java      |   17 +
 .../jvm/storm/trident/util/TridentUtils.java    |   25 +-
 storm-core/src/multilang/py/storm.py            |   18 +
 storm-core/src/multilang/rb/storm.rb            |   18 +
 storm-core/src/storm.thrift                     |   23 +
 .../src/ui/public/css/bootstrap-1.1.0.css       | 1894 --------------
 .../src/ui/public/css/bootstrap-1.4.0.css       | 2469 ++++++++++++++++++
 storm-core/src/ui/public/css/style.css          |   41 +-
 storm-core/src/ui/public/js/bootstrap-twipsy.js |  310 +++
 storm-core/src/ui/public/js/script.js           |   30 +-
 .../test/clj/backtype/storm/clojure_test.clj    |   15 +
 .../test/clj/backtype/storm/cluster_test.clj    |   15 +
 .../test/clj/backtype/storm/config_test.clj     |   15 +
 .../test/clj/backtype/storm/drpc_test.clj       |   15 +
 .../test/clj/backtype/storm/fields_test.clj     |   15 +
 .../test/clj/backtype/storm/grouping_test.clj   |   15 +
 .../clj/backtype/storm/integration_test.clj     |   15 +
 .../clj/backtype/storm/local_state_test.clj     |   15 +
 .../storm/messaging/netty_integration_test.clj  |   59 +
 .../storm/messaging/netty_unit_test.clj         |  112 +
 .../test/clj/backtype/storm/messaging_test.clj  |   17 +-
 .../test/clj/backtype/storm/metrics_test.clj    |   15 +
 .../test/clj/backtype/storm/multilang_test.clj  |   15 +
 .../test/clj/backtype/storm/nimbus_test.clj     |   24 +
 .../test/clj/backtype/storm/scheduler_test.clj  |   15 +
 .../storm/security/auth/AuthUtils_test.clj      |   15 +
 .../storm/security/auth/ReqContext_test.clj     |   15 +
 .../security/auth/SaslTransportPlugin_test.clj  |   15 +
 .../storm/security/auth/ThriftClient_test.clj   |   17 +-
 .../storm/security/auth/ThriftServer_test.clj   |   17 +-
 .../backtype/storm/security/auth/auth_test.clj  |   21 +-
 .../storm/security/auth/jaas_digest.conf        |   18 +
 .../security/auth/jaas_digest_bad_password.conf |   18 +
 .../auth/jaas_digest_missing_client.conf        |   18 +
 .../security/auth/jaas_digest_unknown_user.conf |   18 +
 .../BlowfishTupleSerializer_test.clj            |   15 +
 .../serialization/SerializationFactory_test.clj |   15 +
 .../clj/backtype/storm/serialization_test.clj   |   34 +
 .../clj/backtype/storm/subtopology_test.clj     |   15 +
 .../test/clj/backtype/storm/supervisor_test.clj |   15 +
 .../test/clj/backtype/storm/testing4j_test.clj  |   15 +
 .../test/clj/backtype/storm/tick_tuple_test.clj |   15 +
 .../clj/backtype/storm/transactional_test.clj   |   15 +
 .../test/clj/backtype/storm/tuple_test.clj      |   15 +
 .../test/clj/backtype/storm/utils_test.clj      |   17 +-
 .../clj/backtype/storm/versioned_store_test.clj |   15 +
 .../test/clj/storm/trident/integration_test.clj |   15 +
 .../test/clj/storm/trident/state_test.clj       |   15 +
 .../test/clj/storm/trident/tuple_test.clj       |   15 +
 storm-core/test/clj/zilch/test/mq.clj           |   86 -
 storm-core/test/multilang/fy/bolt.fy            |   16 +
 storm-core/test/multilang/fy/mocks.fy           |   16 +
 storm-core/test/multilang/fy/protocol.fy        |   16 +
 storm-dist/binary/LICENSE                       |  297 +++
 storm-dist/binary/NOTICE                        |   36 +
 storm-dist/binary/pom.xml                       |   78 +
 storm-dist/binary/src/main/assembly/binary.xml  |  105 +
 storm-dist/source/pom.xml                       |   77 +
 storm-dist/source/src/main/assembly/source.xml  |   42 +
 storm-lib/project.clj                           |   18 -
 storm-netty/project.clj                         |   12 -
 .../backtype/storm/messaging/netty/Client.java  |  207 --
 .../backtype/storm/messaging/netty/Context.java |   50 -
 .../storm/messaging/netty/ControlMessage.java   |   51 -
 .../storm/messaging/netty/MessageBatch.java     |  153 --
 .../storm/messaging/netty/MessageDecoder.java   |   68 -
 .../storm/messaging/netty/MessageEncoder.java   |   22 -
 .../backtype/storm/messaging/netty/Server.java  |  120 -
 .../messaging/netty/StormClientHandler.java     |  111 -
 .../netty/StormClientPipelineFactory.java       |   27 -
 .../messaging/netty/StormServerHandler.java     |   59 -
 .../netty/StormServerPipelineFactory.java       |   28 -
 .../storm/messaging/netty_integration_test.clj  |   41 -
 .../storm/messaging/netty_unit_test.clj         |   85 -
 604 files changed, 19113 insertions(+), 7045 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/46c02be8/conf/defaults.yaml
----------------------------------------------------------------------
diff --cc conf/defaults.yaml
index e7a6681,2dbba24..66b2d41
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@@ -18,8 -35,7 +35,8 @@@ storm.zookeeper.retry.intervalceiling.m
  storm.cluster.mode: "distributed" # can be distributed or local
  storm.local.mode.zmq: false
  storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
- storm.messaging.transport: "backtype.storm.messaging.zmq"
 +storm.multilang.serializer: "backtype.storm.multilang.JsonSerializer"
+ storm.messaging.transport: "backtype.storm.messaging.netty.Context"
  
  ### nimbus.* configs are for the master
  nimbus.host: "localhost"

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/46c02be8/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/46c02be8/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/46c02be8/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/46c02be8/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
index 2c3b3a3,5c3e3c9..35a8c94
--- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
@@@ -1,29 -1,44 +1,46 @@@
+ /**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES 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 backtype.storm.Config;
 +import backtype.storm.multilang.ISerializer;
 +import backtype.storm.multilang.BoltMsg;
 +import backtype.storm.multilang.NoOutputException;
 +import backtype.storm.multilang.ShellMsg;
 +import backtype.storm.multilang.SpoutMsg;
  import backtype.storm.task.TopologyContext;
 -import java.io.BufferedReader;
 -import java.io.InputStream;
 -import java.io.InputStreamReader;
 -import java.io.DataOutputStream;
  import java.io.File;
  import java.io.IOException;
 -import java.util.Map;
 +import java.io.InputStream;
 +import java.io.Serializable;
  import java.util.List;
 +import java.util.Map;
  
  import org.apache.commons.io.IOUtils;
 -import org.json.simple.JSONObject;
 -import org.json.simple.JSONValue;
  import org.apache.log4j.Logger;
  
 -public class ShellProcess {
 +public class ShellProcess implements Serializable {
      public static Logger LOG = Logger.getLogger(ShellProcess.class);
 -    private DataOutputStream processIn;
 -    private BufferedReader processOut;
 -    private InputStream processErrorStream;
 -    private Process _subprocess;
 -    private String[] command;
 +    public static Logger ShellLogger;
 +    private Process      _subprocess;
 +    private InputStream  processErrorStream;
 +    private String[]     command;
 +    public ISerializer   serializer;
  
      public ShellProcess(String[] command) {
          this.command = command;


[04/24] git commit: multilang serializer is not handled by stormConf. Renamed Emission and Immission to ShellMsg and BoltMsg respectively. Documented ISerializer interface.

Posted by bo...@apache.org.
multilang serializer is not handled by stormConf. Renamed Emission and Immission to ShellMsg and BoltMsg respectively. Documented ISerializer interface.


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

Branch: refs/heads/master
Commit: 3219a4f05f938f4ce5bdbb9f006e49cbec5ee89f
Parents: 3701bfc
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Tue Oct 8 13:52:00 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Tue Oct 8 13:52:00 2013 +0200

----------------------------------------------------------------------
 conf/defaults.yaml                              |   1 +
 storm-core/src/jvm/backtype/storm/Config.java   | 129 +++++++++--------
 .../jvm/backtype/storm/multilang/BoltMsg.java   |  63 +++++++++
 .../jvm/backtype/storm/multilang/Emission.java  |  79 -----------
 .../backtype/storm/multilang/ISerializer.java   |  59 ++++++--
 .../jvm/backtype/storm/multilang/Immission.java |  52 -------
 .../storm/multilang/JsonSerializer.java         | 138 ++++++++++---------
 .../storm/multilang/NoOutputException.java      |  19 ++-
 .../jvm/backtype/storm/multilang/ShellMsg.java  |  96 +++++++++++++
 .../jvm/backtype/storm/multilang/SpoutMsg.java  |  44 +++---
 .../jvm/backtype/storm/spout/ShellSpout.java    |  30 ++--
 .../src/jvm/backtype/storm/task/ShellBolt.java  |  65 ++++-----
 .../jvm/backtype/storm/utils/ShellProcess.java  |  95 ++++++++-----
 13 files changed, 489 insertions(+), 381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index b17e445..92e65d6 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -19,6 +19,7 @@ storm.cluster.mode: "distributed" # can be distributed or local
 storm.local.mode.zmq: false
 storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
 storm.messaging.transport: "backtype.storm.messaging.zmq"
+storm.multilang.serializer: "backtype.storm.multilang.JsonSerializer"
 
 ### nimbus.* configs are for the master
 nimbus.host: "localhost"

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/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 564ad0d..fff1573 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -10,11 +10,11 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Topology configs are specified as a plain old map. This class provides a 
- * convenient way to create a topology config map by providing setter methods for 
- * all the configs that can be set. It also makes it easier to do things like add 
+ * Topology configs are specified as a plain old map. This class provides a
+ * convenient way to create a topology config map by providing setter methods for
+ * all the configs that can be set. It also makes it easier to do things like add
  * serializations.
- * 
+ *
  * <p>This class also provides constants for all the configurations possible on
  * a Storm cluster and Storm topology. Each constant is paired with a schema
  * that defines the validity criterion of the corresponding field. Default
@@ -22,11 +22,18 @@ import java.util.Map;
  *
  * <p>Note that you may put other configurations in any of the configs. Storm
  * will ignore anything it doesn't recognize, but your topologies are free to make
- * use of them by reading them in the prepare method of Bolts or the open method of 
+ * use of them by reading them in the prepare method of Bolts or the open method of
  * Spouts.</p>
  */
 public class Config extends HashMap<String, Object> {
     /**
+     * The serializer for communication between shell components and non-JVM
+     * processes
+     */
+    public static final String STORM_MULTILANG_SERIALIZER = "storm.multilang.serializer";
+    public static final Object STORM_MULTILANG_SERIALIZER_SCHEMA = String.class;
+
+    /**
      * The transporter for communication among Storm tasks
      */
     public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport";
@@ -35,25 +42,25 @@ public class Config extends HashMap<String, Object> {
     /**
      * Netty based messaging: The buffer size for send/recv buffer
      */
-    public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; 
+    public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size";
     public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = Number.class;
 
     /**
      * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible
      */
-    public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; 
+    public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries";
     public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = Number.class;
 
     /**
-     * Netty based messaging: The min # of milliseconds that a peer will wait. 
+     * Netty based messaging: The min # of milliseconds that a peer will wait.
      */
-    public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; 
+    public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms";
     public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = Number.class;
 
     /**
-     * Netty based messaging: The max # of milliseconds that a peer will wait. 
+     * Netty based messaging: The max # of milliseconds that a peer will wait.
      */
-    public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; 
+    public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms";
     public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = Number.class;
 
     /**
@@ -78,7 +85,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A global task scheduler used to assign topologies's tasks to supervisors' wokers.
-     * 
+     *
      * If this is not set, a default system scheduler will be used.
      */
     public static final String STORM_SCHEDULER = "storm.scheduler";
@@ -91,9 +98,9 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class;
 
     /**
-     * The hostname the supervisors/workers should report to nimbus. If unset, Storm will 
+     * The hostname the supervisors/workers should report to nimbus. If unset, Storm will
      * get the hostname to report by calling <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
-     * 
+     *
      * You should set this config when you dont have a DNS which supervisors/workers
      * can utilize to find each other based on hostname got from calls to
      * <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
@@ -108,16 +115,16 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
-     * The serializer class for ListDelegate (tuple payload). 
+     * The serializer class for ListDelegate (tuple payload).
      * The default serializer will be ListDelegateSerializer
      */
     public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer";
     public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class;
 
     /**
-     * Whether or not to use ZeroMQ for messaging in local mode. If this is set 
-     * to false, then Storm will use a pure-Java messaging system. The purpose 
-     * of this flag is to make it easy to run Storm in local mode by eliminating 
+     * Whether or not to use ZeroMQ for messaging in local mode. If this is set
+     * to false, then Storm will use a pure-Java messaging system. The purpose
+     * of this flag is to make it easy to run Storm in local mode by eliminating
      * the need for native dependencies, which can be difficult to install.
      *
      * Defaults to false.
@@ -255,7 +262,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = Number.class;
 
     /**
-     * Whether or not nimbus should reassign tasks if it detects that a task goes down. 
+     * Whether or not nimbus should reassign tasks if it detects that a task goes down.
      * Defaults to true, and it's not recommended to change this value.
      */
     public static final String NIMBUS_REASSIGN = "nimbus.reassign";
@@ -319,19 +326,19 @@ public class Config extends HashMap<String, Object> {
     public static final Object DRPC_PORT_SCHEMA = Number.class;
 
     /**
-     * DRPC thrift server worker threads 
+     * DRPC thrift server worker threads
      */
     public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
     public static final Object DRPC_WORKER_THREADS_SCHEMA = Number.class;
 
     /**
-     * DRPC thrift server queue size 
+     * DRPC thrift server queue size
      */
     public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
     public static final Object DRPC_QUEUE_SIZE_SCHEMA = Number.class;
 
     /**
-     * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back. 
+     * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.
      */
     public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
     public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = Number.class;
@@ -479,8 +486,8 @@ public class Config extends HashMap<String, Object> {
     /**
      * How many instances to create for a spout/bolt. A task runs on a thread with zero or more
      * other tasks for the same spout/bolt. The number of tasks for a spout/bolt is always
-     * the same throughout the lifetime of a topology, but the number of executors (threads) for 
-     * a spout/bolt can change over time. This allows a topology to scale to more or less resources 
+     * the same throughout the lifetime of a topology, but the number of executors (threads) for
+     * a spout/bolt can change over time. This allows a topology to scale to more or less resources
      * without redeploying the topology or violating the constraints of Storm (such as a fields grouping
      * guaranteeing that the same value goes to the same task).
      */
@@ -519,8 +526,8 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A list of classes that customize storm's kryo instance during start-up.
-     * Each listed class name must implement IKryoDecorator. During start-up the 
-     * listed class is instantiated with 0 arguments, then its 'decorate' method 
+     * Each listed class name must implement IKryoDecorator. During start-up the
+     * listed class is instantiated with 0 arguments, then its 'decorate' method
      * is called with storm's kryo instance as the only argument.
      */
     public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators";
@@ -550,7 +557,7 @@ public class Config extends HashMap<String, Object> {
 
     /*
      * A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format).
-     * Each listed class will be routed all the metrics data generated by the storm metrics API. 
+     * Each listed class will be routed all the metrics data generated by the storm metrics API.
      * Each listed class maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable.
      */
     public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register";
@@ -566,24 +573,24 @@ public class Config extends HashMap<String, Object> {
 
 
     /**
-     * The maximum number of tuples that can be pending on a spout task at any given time. 
-     * This config applies to individual tasks, not to spouts or topologies as a whole. 
-     * 
+     * The maximum number of tuples that can be pending on a spout task at any given time.
+     * This config applies to individual tasks, not to spouts or topologies as a whole.
+     *
      * A pending tuple is one that has been emitted from a spout but has not been acked or failed yet.
-     * Note that this config parameter has no effect for unreliable spouts that don't tag 
+     * Note that this config parameter has no effect for unreliable spouts that don't tag
      * their tuples with a message id.
      */
-    public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; 
+    public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending";
     public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = Number.class;
 
     /**
      * A class that implements a strategy for what to do when a spout needs to wait. Waiting is
      * triggered in one of two conditions:
-     * 
+     *
      * 1. nextTuple emits no tuples
      * 2. The spout has hit maxSpoutPending and can't emit any more tuples
      */
-    public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy"; 
+    public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy";
     public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class;
 
     /**
@@ -606,7 +613,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA = Number.class;
 
     /**
-     * The time period that builtin metrics data in bucketed into. 
+     * The time period that builtin metrics data in bucketed into.
      */
     public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs";
     public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = Number.class;
@@ -633,7 +640,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * A list of task hooks that are automatically added to every spout and bolt in the topology. An example
-     * of when you'd do this is to add a hook that integrates with your internal 
+     * of when you'd do this is to add a hook that integrates with your internal
      * monitoring system. These hooks are instantiated using the zero-arg constructor.
      */
     public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks";
@@ -647,7 +654,7 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator;
 
     /**
-     * The maximum number of messages to batch from the thread receiving off the network to the 
+     * The maximum number of messages to batch from the thread receiving off the network to the
      * executor queues. Must be a power of 2.
      */
     public static final String TOPOLOGY_RECEIVER_BUFFER_SIZE="topology.receiver.buffer.size";
@@ -681,14 +688,14 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class;
 
    /**
-    * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed 
+    * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
     * via the TopologyContext.
     */
     public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size";
     public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = Number.class;
 
     /**
-     * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, 
+     * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
      * an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
      * reported to Zookeeper per task for every 10 second interval of time.
      */
@@ -757,9 +764,9 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
-     * for the java.library.path value. java.library.path tells the JVM where 
+     * for the java.library.path value. java.library.path tells the JVM where
      * to look for native libraries. It is necessary to set this config correctly since
-     * Storm uses the ZeroMQ and JZMQ native libs. 
+     * Storm uses the ZeroMQ and JZMQ native libs.
      */
     public static final String JAVA_LIBRARY_PATH = "java.library.path";
     public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class;
@@ -781,17 +788,17 @@ public class Config extends HashMap<String, Object> {
 
     public static void setDebug(Map conf, boolean isOn) {
         conf.put(Config.TOPOLOGY_DEBUG, isOn);
-    } 
+    }
 
     public void setDebug(boolean isOn) {
         setDebug(this, isOn);
     }
-    
+
     @Deprecated
     public void setOptimize(boolean isOn) {
         put(Config.TOPOLOGY_OPTIMIZE, isOn);
-    } 
-    
+    }
+
     public static void setNumWorkers(Map conf, int workers) {
         conf.put(Config.TOPOLOGY_WORKERS, workers);
     }
@@ -807,7 +814,7 @@ public class Config extends HashMap<String, Object> {
     public void setNumAckers(int numExecutors) {
         setNumAckers(this, numExecutors);
     }
-    
+
     public static void setMessageTimeoutSecs(Map conf, int secs) {
         conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs);
     }
@@ -815,7 +822,7 @@ public class Config extends HashMap<String, Object> {
     public void setMessageTimeoutSecs(int secs) {
         setMessageTimeoutSecs(this, secs);
     }
-    
+
     public static void registerSerialization(Map conf, Class klass) {
         getRegisteredSerializations(conf).add(klass.getName());
     }
@@ -823,17 +830,17 @@ public class Config extends HashMap<String, Object> {
     public void registerSerialization(Class klass) {
         registerSerialization(this, klass);
     }
-    
+
     public static void registerSerialization(Map conf, Class klass, Class<? extends Serializer> serializerClass) {
         Map<String, String> register = new HashMap<String, String>();
         register.put(klass.getName(), serializerClass.getName());
-        getRegisteredSerializations(conf).add(register);        
+        getRegisteredSerializations(conf).add(register);
     }
 
     public void registerSerialization(Class klass, Class<? extends Serializer> serializerClass) {
         registerSerialization(this, klass, serializerClass);
     }
-    
+
     public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) {
         HashMap m = new HashMap();
         m.put("class", klass.getCanonicalName());
@@ -861,7 +868,7 @@ public class Config extends HashMap<String, Object> {
     public void registerDecorator(Class<? extends IKryoDecorator> klass) {
         registerDecorator(this, klass);
     }
-    
+
     public static void setKryoFactory(Map conf, Class<? extends IKryoFactory> klass) {
         conf.put(Config.TOPOLOGY_KRYO_FACTORY, klass.getName());
     }
@@ -877,7 +884,7 @@ public class Config extends HashMap<String, Object> {
     public void setSkipMissingKryoRegistrations(boolean skip) {
        setSkipMissingKryoRegistrations(this, skip);
     }
-    
+
     public static void setMaxTaskParallelism(Map conf, int max) {
         conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max);
     }
@@ -885,7 +892,7 @@ public class Config extends HashMap<String, Object> {
     public void setMaxTaskParallelism(int max) {
         setMaxTaskParallelism(this, max);
     }
-    
+
     public static void setMaxSpoutPending(Map conf, int max) {
         conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max);
     }
@@ -893,23 +900,23 @@ public class Config extends HashMap<String, Object> {
     public void setMaxSpoutPending(int max) {
         setMaxSpoutPending(this, max);
     }
-    
+
     public static void setStatsSampleRate(Map conf, double rate) {
         conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, rate);
-    }    
+    }
 
     public void setStatsSampleRate(double rate) {
         setStatsSampleRate(this, rate);
-    }    
+    }
 
     public static void setFallBackOnJavaSerialization(Map conf, boolean fallback) {
         conf.put(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION, fallback);
-    }    
+    }
 
     public void setFallBackOnJavaSerialization(boolean fallback) {
         setFallBackOnJavaSerialization(this, fallback);
-    }    
-    
+    }
+
     private static List getRegisteredSerializations(Map conf) {
         List ret;
         if(!conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) {
@@ -920,13 +927,13 @@ public class Config extends HashMap<String, Object> {
         conf.put(Config.TOPOLOGY_KRYO_REGISTER, ret);
         return ret;
     }
-    
+
     private static List getRegisteredDecorators(Map conf) {
         List ret;
         if(!conf.containsKey(Config.TOPOLOGY_KRYO_DECORATORS)) {
             ret = new ArrayList();
         } else {
-            ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS));            
+            ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS));
         }
         conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret);
         return ret;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java b/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java
new file mode 100644
index 0000000..1d6bd1d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/BoltMsg.java
@@ -0,0 +1,63 @@
+package backtype.storm.multilang;
+
+import java.util.List;
+
+/**
+ * BoltMsg is an object that represents the data sent from a shell component to
+ * a bolt process that implements a multi-language protocol. It is the union of
+ * all data types that a bolt can receive from Storm.
+ *
+ * <p>
+ * BoltMsgs are objects sent to the ISerializer interface, for serialization
+ * according to the wire protocol implemented by the serializer. The BoltMsg
+ * class allows for a decoupling between the serialized representation of the
+ * data and the data itself.
+ * </p>
+ */
+public class BoltMsg {
+    private String id;
+    private String comp;
+    private String stream;
+    private long task;
+    private List<Object> tuple;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getComp() {
+        return comp;
+    }
+
+    public void setComp(String comp) {
+        this.comp = comp;
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    public void setStream(String stream) {
+        this.stream = stream;
+    }
+
+    public long getTask() {
+        return task;
+    }
+
+    public void setTask(long task) {
+        this.task = task;
+    }
+
+    public List<Object> getTuple() {
+        return tuple;
+    }
+
+    public void setTuple(List<Object> tuple) {
+        this.tuple = tuple;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/Emission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Emission.java b/storm-core/src/jvm/backtype/storm/multilang/Emission.java
deleted file mode 100644
index d477d8b..0000000
--- a/storm-core/src/jvm/backtype/storm/multilang/Emission.java
+++ /dev/null
@@ -1,79 +0,0 @@
-package backtype.storm.multilang;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Emission is an object that represents the data sent to a shell component
- * from a process that implements a multi-language protocol. It is the union
- * of all data types that a component can send to Storm.
- *
- * <p> Emissions are objects received from the ISerializer interface, after
- * the serializer has deserialized the data from the underlying wire protocol.
- * The Emission class allows for a decoupling between the serialized
- * representation of the data and the data itself.</p>
- */
-public class Emission {
-	private String command;
-	private String id;
-	private List<String> anchors;
-	private String stream;
-	private long task;
-	private String msg;
-	private List<Object> tuple;
-
-	public String getCommand() {
-		return command;
-	}
-	public void setCommand(String command) {
-		this.command = command;
-	}
-	public String getId() {
-		return id;
-	}
-	public void setId(String id) {
-		this.id = id;
-	}
-	public List<String> getAnchors() {
-		return anchors;
-	}
-	public void setAnchors(List<String> anchors) {
-		this.anchors = anchors;
-	}
-	public void addAnchor(String anchor) {
-		if (anchors == null) {
-			anchors = new ArrayList<String>();
-		}
-		this.anchors.add(anchor);
-	}
-	public String getStream() {
-		return stream;
-	}
-	public void setStream(String stream) {
-		this.stream = stream;
-	}
-	public long getTask() {
-		return task;
-	}
-	public void setTask(long task) {
-		this.task = task;
-	}
-	public String getMsg() {
-		return msg;
-	}
-	public void setMsg(String msg) {
-		this.msg = msg;
-	}
-	public List<Object> getTuple() {
-		return tuple;
-	}
-	public void setTuple(List<Object> tuple) {
-		this.tuple = tuple;
-	}
-	public void addTuple(Object tuple) {
-		if (this.tuple == null) {
-			this.tuple = new ArrayList<Object>();
-		}
-		this.tuple.add(tuple);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
index 691b432..a6ccfd4 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/ISerializer.java
@@ -10,15 +10,56 @@ import java.util.Map;
 import backtype.storm.task.TopologyContext;
 
 /**
- * The ISerializer interface describes the methods that an object should implement
- * to provide serialization and de-serialization capabilities to non-JVM
- * language components.
+ * The ISerializer interface describes the methods that an object should
+ * implement to provide serialization and de-serialization capabilities to
+ * non-JVM language components.
  */
 public interface ISerializer extends Serializable {
-	void initialize (OutputStream processIn, InputStream processOut);
-	Number connect (Map conf, TopologyContext context) throws IOException, NoOutputException;
-	Emission readEmission() throws IOException, NoOutputException;
-	void writeImmission(Immission immission) throws IOException;
-	void writeSpoutMsg(SpoutMsg msg) throws IOException;
-	void writeTaskIds(List<Integer> taskIds) throws IOException;
+
+    /**
+     * This method sets the input and output streams of the serializer
+     *
+     * @param processIn output stream to non-JVM component
+     * @param processOut input stream from non-JVM component
+     */
+    void initialize(OutputStream processIn, InputStream processOut);
+
+    /**
+     * This method transmits the Storm config to the non-JVM process and
+     * receives its pid.
+     *
+     * @param conf storm configuration
+     * @param context topology context
+     * @return process pid
+     */
+    Number connect(Map conf, TopologyContext context) throws IOException,
+            NoOutputException;
+
+    /**
+     * This method receives a shell message from the non-JVM process
+     *
+     * @return shell message
+     */
+    ShellMsg readShellMsg() throws IOException, NoOutputException;
+
+    /**
+     * This method sends a bolt message to a non-JVM bolt process
+     *
+     * @param msg bolt message
+     */
+    void writeBoltMsg(BoltMsg msg) throws IOException;
+
+    /**
+     * This method sends a spout message to a non-JVM spout process
+     *
+     * @param msg spout message
+     */
+    void writeSpoutMsg(SpoutMsg msg) throws IOException;
+
+    /**
+     * This method sends a list of task IDs to a non-JVM bolt process
+     *
+     * @param taskIds list of task IDs
+     */
+    void writeTaskIds(List<Integer> taskIds) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/Immission.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/Immission.java b/storm-core/src/jvm/backtype/storm/multilang/Immission.java
deleted file mode 100644
index 968f728..0000000
--- a/storm-core/src/jvm/backtype/storm/multilang/Immission.java
+++ /dev/null
@@ -1,52 +0,0 @@
-package backtype.storm.multilang;
-
-import java.util.List;
-
-/**
- * Immission is an object that represents the data sent from a shell component
- * to a process that implements a multi-language protocol. It is the union
- * of all data types that a component can receive from Storm.
- *
- * <p> Immissions are objects sent to the ISerializer interface, for
- * serialization according to the wire protocol implemented by the serializer.
- * The Immission class allows for a decoupling between the serialized
- * representation of the data and the data itself.</p>
- */
-public class Immission {
-	private String id;
-	private String comp;
-	private String stream;
-	private long task;
-	private List<Object> tuple;
-
-	public String getId() {
-		return id;
-	}
-	public void setId(String id) {
-		this.id = id;
-	}
-	public String getComp() {
-		return comp;
-	}
-	public void setComp(String comp) {
-		this.comp = comp;
-	}
-	public String getStream() {
-		return stream;
-	}
-	public void setStream(String stream) {
-		this.stream = stream;
-	}
-	public long getTask() {
-		return task;
-	}
-	public void setTask(long task) {
-		this.task = task;
-	}
-	public List<Object> getTuple() {
-		return tuple;
-	}
-	public void setTuple(List<Object> tuple) {
-		this.tuple = tuple;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
index ba574e5..ceb8cb3 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -22,126 +22,128 @@ import backtype.storm.utils.Utils;
  * JsonSerializer implements the JSON multilang protocol.
  */
 public class JsonSerializer implements ISerializer {
-	private DataOutputStream processIn;
+    private DataOutputStream processIn;
     private BufferedReader processOut;
 
-	public void initialize (OutputStream processIn, InputStream processOut) {
-		this.processIn = new DataOutputStream(processIn);;
-		this.processOut = new BufferedReader(new InputStreamReader(processOut));
-	}
+    public void initialize(OutputStream processIn, InputStream processOut) {
+        this.processIn = new DataOutputStream(processIn);
+        this.processOut = new BufferedReader(new InputStreamReader(processOut));
+    }
 
-	public Number connect (Map conf, TopologyContext context) throws IOException, NoOutputException {
-		JSONObject setupInfo = new JSONObject();
+    public Number connect(Map conf, TopologyContext context)
+            throws IOException, NoOutputException {
+        JSONObject setupInfo = new JSONObject();
         setupInfo.put("pidDir", context.getPIDDir());
         setupInfo.put("conf", conf);
         setupInfo.put("context", context);
-    	writeMessage(setupInfo);
+        writeMessage(setupInfo);
 
-    	Number pid = (Number)((JSONObject)readMessage()).get("pid");
+        Number pid = (Number) ((JSONObject) readMessage()).get("pid");
         return pid;
-	}
-
-	public void writeImmission(Immission immission) throws IOException {
-		JSONObject obj = new JSONObject();
-        obj.put("id", immission.getId());
-        obj.put("comp", immission.getComp());
-        obj.put("stream", immission.getStream());
-        obj.put("task", immission.getTask());
-        obj.put("tuple", immission.getTuple());
+    }
+
+    public void writeBoltMsg(BoltMsg boltMsg) throws IOException {
+        JSONObject obj = new JSONObject();
+        obj.put("id", boltMsg.getId());
+        obj.put("comp", boltMsg.getComp());
+        obj.put("stream", boltMsg.getStream());
+        obj.put("task", boltMsg.getTask());
+        obj.put("tuple", boltMsg.getTuple());
         writeMessage(obj);
-	}
+    }
 
-	public void writeSpoutMsg(SpoutMsg msg) throws IOException {
-		JSONObject obj = new JSONObject();
+    public void writeSpoutMsg(SpoutMsg msg) throws IOException {
+        JSONObject obj = new JSONObject();
         obj.put("command", msg.getCommand());
         obj.put("id", msg.getId());
         writeMessage(obj);
-	}
+    }
 
-	public void writeTaskIds(List<Integer> taskIds) throws IOException {
+    public void writeTaskIds(List<Integer> taskIds) throws IOException {
         writeMessage(taskIds);
-	}
+    }
 
-	private void writeMessage(Object msg) throws IOException {
+    private void writeMessage(Object msg) throws IOException {
         writeString(JSONValue.toJSONString(msg));
     }
 
-	private void writeString(String str) throws IOException {
+    private void writeString(String str) throws IOException {
         byte[] strBytes = str.getBytes("UTF-8");
         processIn.write(strBytes, 0, strBytes.length);
         processIn.writeBytes("\nend\n");
         processIn.flush();
     }
 
-	public Emission readEmission() throws IOException, NoOutputException {
-        JSONObject msg = (JSONObject)readMessage();
-    	Emission emission = new Emission();
+    public ShellMsg readShellMsg() throws IOException, NoOutputException {
+        JSONObject msg = (JSONObject) readMessage();
+        ShellMsg shellMsg = new ShellMsg();
 
-    	String command = (String) msg.get("command");
-        emission.setCommand(command);
+        String command = (String) msg.get("command");
+        shellMsg.setCommand(command);
 
-    	String stream = (String) msg.get("stream");
-        if(stream == null) stream = Utils.DEFAULT_STREAM_ID;
-        emission.setStream(stream);
+        String stream = (String) msg.get("stream");
+        if (stream == null)
+            stream = Utils.DEFAULT_STREAM_ID;
+        shellMsg.setStream(stream);
 
         Object taskObj = msg.get("task");
         if (taskObj != null) {
-        	emission.setTask((Long) taskObj);
+            shellMsg.setTask((Long) taskObj);
         }
 
-        emission.setTuple((List) msg.get("tuple"));
+        shellMsg.setTuple((List) msg.get("tuple"));
 
         List<Tuple> anchors = new ArrayList<Tuple>();
         Object anchorObj = msg.get("anchors");
-        if(anchorObj!=null) {
-            if(anchorObj instanceof String) {
+        if (anchorObj != null) {
+            if (anchorObj instanceof String) {
                 anchorObj = Arrays.asList(anchorObj);
             }
-            for(Object o: (List) anchorObj) {
-                emission.addAnchor((String)o);
+            for (Object o : (List) anchorObj) {
+                shellMsg.addAnchor((String) o);
             }
         }
 
-        return emission;
+        return shellMsg;
     }
 
-	private Object readMessage() throws IOException, NoOutputException {
-		String string = readString();
+    private Object readMessage() throws IOException, NoOutputException {
+        String string = readString();
         Object msg = JSONValue.parse(string);
         if (msg != null) {
-        	return msg;
+            return msg;
         } else {
-        	throw new IOException("unable to parse: " + string);
+            throw new IOException("unable to parse: " + string);
         }
-	}
+    }
 
     private String readString() throws IOException, NoOutputException {
         StringBuilder line = new StringBuilder();
 
-        //synchronized (processOut) {
-            while (true) {
-                String subline = processOut.readLine();
-                if(subline==null) {
-                    StringBuilder errorMessage = new StringBuilder();
-                    errorMessage.append("Pipe to subprocess seems to be broken!");
-                    if (line.length() == 0) {
-                        errorMessage.append(" No output read.\n");
-                    }
-                    else {
-                        errorMessage.append(" Currently read output: " + line.toString() + "\n");
-                    }
-                    errorMessage.append("Serializer Exception:\n");
-                    throw new NoOutputException(errorMessage.toString());
-                }
-                if(subline.equals("end")) {
-                    break;
-                }
-                if(line.length()!=0) {
-                    line.append("\n");
+        // synchronized (processOut) {
+        while (true) {
+            String subline = processOut.readLine();
+            if (subline == null) {
+                StringBuilder errorMessage = new StringBuilder();
+                errorMessage.append("Pipe to subprocess seems to be broken!");
+                if (line.length() == 0) {
+                    errorMessage.append(" No output read.\n");
+                } else {
+                    errorMessage.append(" Currently read output: "
+                            + line.toString() + "\n");
                 }
-                line.append(subline);
+                errorMessage.append("Serializer Exception:\n");
+                throw new NoOutputException(errorMessage.toString());
             }
-        //}
+            if (subline.equals("end")) {
+                break;
+            }
+            if (line.length() != 0) {
+                line.append("\n");
+            }
+            line.append(subline);
+        }
+        // }
         return line.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
index 8ffb7dd..fd5ef72 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/NoOutputException.java
@@ -5,8 +5,19 @@ package backtype.storm.multilang;
  * non-JVM process.
  */
 public class NoOutputException extends Exception {
-	public NoOutputException() { super(); }
-	public NoOutputException(String message) { super(message); }
-	public NoOutputException(String message, Throwable cause) { super(message, cause); }
-	public NoOutputException(Throwable cause) { super(cause); }
+    public NoOutputException() {
+        super();
+    }
+
+    public NoOutputException(String message) {
+        super(message);
+    }
+
+    public NoOutputException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NoOutputException(Throwable cause) {
+        super(cause);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
new file mode 100644
index 0000000..d56ba78
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/multilang/ShellMsg.java
@@ -0,0 +1,96 @@
+package backtype.storm.multilang;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * ShellMsg is an object that represents the data sent to a shell component from
+ * a process that implements a multi-language protocol. It is the union of all
+ * data types that a component can send to Storm.
+ *
+ * <p>
+ * ShellMsgs are objects received from the ISerializer interface, after the
+ * serializer has deserialized the data from the underlying wire protocol. The
+ * ShellMsg class allows for a decoupling between the serialized representation
+ * of the data and the data itself.
+ * </p>
+ */
+public class ShellMsg {
+    private String command;
+    private String id;
+    private List<String> anchors;
+    private String stream;
+    private long task;
+    private String msg;
+    private List<Object> tuple;
+
+    public String getCommand() {
+        return command;
+    }
+
+    public void setCommand(String command) {
+        this.command = command;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public List<String> getAnchors() {
+        return anchors;
+    }
+
+    public void setAnchors(List<String> anchors) {
+        this.anchors = anchors;
+    }
+
+    public void addAnchor(String anchor) {
+        if (anchors == null) {
+            anchors = new ArrayList<String>();
+        }
+        this.anchors.add(anchor);
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    public void setStream(String stream) {
+        this.stream = stream;
+    }
+
+    public long getTask() {
+        return task;
+    }
+
+    public void setTask(long task) {
+        this.task = task;
+    }
+
+    public String getMsg() {
+        return msg;
+    }
+
+    public void setMsg(String msg) {
+        this.msg = msg;
+    }
+
+    public List<Object> getTuple() {
+        return tuple;
+    }
+
+    public void setTuple(List<Object> tuple) {
+        this.tuple = tuple;
+    }
+
+    public void addTuple(Object tuple) {
+        if (this.tuple == null) {
+            this.tuple = new ArrayList<Object>();
+        }
+        this.tuple.add(tuple);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
index c8063eb..9b5ffce 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/SpoutMsg.java
@@ -1,28 +1,34 @@
 package backtype.storm.multilang;
 
 /**
- * SpoutMsg is an object that represents the data sent from a shell spout
- * to a process that implements a multi-language spout. The SpoutMsg is used
- * to send a "next", "ack" or "fail" message to a spout.
+ * SpoutMsg is an object that represents the data sent from a shell spout to a
+ * process that implements a multi-language spout. The SpoutMsg is used to send
+ * a "next", "ack" or "fail" message to a spout.
  *
- * <p> Spout messages are objects sent to the ISerializer interface, for
+ * <p>
+ * Spout messages are objects sent to the ISerializer interface, for
  * serialization according to the wire protocol implemented by the serializer.
  * The SpoutMsg class allows for a decoupling between the serialized
- * representation of the data and the data itself.</p>
+ * representation of the data and the data itself.
+ * </p>
  */
 public class SpoutMsg {
-	private String command;
-	private String id;
-	public String getCommand() {
-		return command;
-	}
-	public void setCommand(String command) {
-		this.command = command;
-	}
-	public String getId() {
-		return id;
-	}
-	public void setId(String id) {
-		this.id = id;
-	}
+    private String command;
+    private String id;
+
+    public String getCommand() {
+        return command;
+    }
+
+    public void setCommand(String command) {
+        this.command = command;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index c9ef682..72cebe2 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -1,13 +1,10 @@
 package backtype.storm.spout;
 
 import backtype.storm.generated.ShellComponent;
-import backtype.storm.multilang.Emission;
-import backtype.storm.multilang.ISerializer;
-import backtype.storm.multilang.JsonSerializer;
+import backtype.storm.multilang.ShellMsg;
 import backtype.storm.multilang.SpoutMsg;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.utils.ShellProcess;
-import backtype.storm.utils.Utils;
 import java.util.Map;
 import java.util.List;
 import java.io.IOException;
@@ -25,22 +22,16 @@ public class ShellSpout implements ISpout {
 
     public ShellSpout(ShellComponent component) {
         this(component.get_execution_command(), component.get_script());
-        _process = new ShellProcess(new JsonSerializer(), _command);
     }
 
     public ShellSpout(String... command) {
         _command = command;
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellSpout(ISerializer serializer, String... command) {
-        _command = command;
-        _process = new ShellProcess(serializer, _command);
     }
 
     public void open(Map stormConf, TopologyContext context,
                      SpoutOutputCollector collector) {
         _collector = collector;
+        _process = new ShellProcess(_command);
 
         Number subpid = _process.launch(stormConf, context);
         LOG.info("Launched subprocess with pid " + subpid);
@@ -82,23 +73,24 @@ public class ShellSpout implements ISpout {
             _process.writeSpoutMsg(spoutMsg);
 
             while (true) {
-                Emission emission = _process.readEmission();
-                String command = emission.getCommand();
+                ShellMsg shellMsg = _process.readShellMsg();
+                String command = shellMsg.getCommand();
                 if (command.equals("sync")) {
                     return;
                 } else if (command.equals("log")) {
-                    String msg = emission.getMsg();
+                    String msg = shellMsg.getMsg();
                     LOG.info("Shell msg: " + msg);
                 } else if (command.equals("emit")) {
-                    String stream = emission.getStream();
-                    Long task = emission.getTask();
-                    List<Object> tuple = emission.getTuple();
-                    Object messageId = emission.getId();
+                    String stream = shellMsg.getStream();
+                    Long task = shellMsg.getTask();
+                    List<Object> tuple = shellMsg.getTuple();
+                    Object messageId = shellMsg.getId();
                     if (task == 0) {
                         List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
                         _process.writeTaskIds(outtasks);
                     } else {
-                        _collector.emitDirect((int)task.longValue(), stream, tuple, messageId);
+                        _collector.emitDirect((int) task.longValue(), stream,
+                                tuple, messageId);
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index 36585e3..6a6c305 100644
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -3,10 +3,9 @@ package backtype.storm.task;
 import backtype.storm.generated.ShellComponent;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.utils.ShellProcess;
-import backtype.storm.multilang.Emission;
-import backtype.storm.multilang.ISerializer;
-import backtype.storm.multilang.Immission;
-import backtype.storm.multilang.JsonSerializer;
+import backtype.storm.multilang.BoltMsg;
+import backtype.storm.multilang.ShellMsg;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
@@ -62,23 +61,17 @@ public class ShellBolt implements IBolt {
 
     public ShellBolt(ShellComponent component) {
         this(component.get_execution_command(), component.get_script());
-        _process = new ShellProcess(new JsonSerializer(), _command);
     }
 
     public ShellBolt(String... command) {
         _command = command;
-        _process = new ShellProcess(new JsonSerializer(), _command);
-    }
-
-    public ShellBolt(ISerializer serializer, String... command) {
-        _command = command;
-        _process = new ShellProcess(serializer, _command);
     }
 
     public void prepare(Map stormConf, TopologyContext context,
                         final OutputCollector collector) {
         _rand = new Random();
         _collector = collector;
+        _process = new ShellProcess(_command);
 
         //subprocesses must send their pid first thing
         Number subpid = _process.launch(stormConf, context);
@@ -89,20 +82,20 @@ public class ShellBolt implements IBolt {
             public void run() {
                 while (_running) {
                     try {
-                        Emission emission = _process.readEmission();
+                        ShellMsg shellMsg = _process.readShellMsg();
 
-                        String command = emission.getCommand();
+                        String command = shellMsg.getCommand();
                         if(command.equals("ack")) {
-                            handleAck(emission.getId());
+                            handleAck(shellMsg.getId());
                         } else if (command.equals("fail")) {
-                            handleFail(emission.getId());
+                            handleFail(shellMsg.getId());
                         } else if (command.equals("error")) {
-                            handleError(emission.getMsg());
+                            handleError(shellMsg.getMsg());
                         } else if (command.equals("log")) {
-                            String msg = emission.getMsg();
+                            String msg = shellMsg.getMsg();
                             LOG.info("Shell msg: " + msg);
                         } else if (command.equals("emit")) {
-                            handleEmit(emission);
+                            handleEmit(shellMsg);
                         }
                     } catch (InterruptedException e) {
                     } catch (Throwable t) {
@@ -118,12 +111,13 @@ public class ShellBolt implements IBolt {
             public void run() {
                 while (_running) {
                     try {
-                    	// FIXME: This can either be TaskIds or Immissions
                         Object write = _pendingWrites.poll(1, SECONDS);
-                        if (write instanceof Immission) {
-                            _process.writeImmission((Immission)write);
+                        if (write instanceof BoltMsg) {
+                            _process.writeBoltMsg((BoltMsg)write);
                         } else if (write instanceof List<?>) {
-
+                            _process.writeTaskIds((List<Integer>)write);
+                        } else {
+                            throw new RuntimeException("Cannot write object to bolt:\n" + write.toString());
                         }
                     } catch (InterruptedException e) {
                     } catch (Throwable t) {
@@ -145,14 +139,14 @@ public class ShellBolt implements IBolt {
         String genId = Long.toString(_rand.nextLong());
         _inputs.put(genId, input);
         try {
-            Immission immission = new Immission();
-            immission.setId(genId);
-            immission.setComp(input.getSourceComponent());
-            immission.setStream(input.getSourceStreamId());
-            immission.setTask(input.getSourceTask());
-            immission.setTuple(input.getValues());
-
-            _pendingWrites.put(immission);
+            BoltMsg boltMsg = new BoltMsg();
+            boltMsg.setId(genId);
+            boltMsg.setComp(input.getSourceComponent());
+            boltMsg.setStream(input.getSourceStreamId());
+            boltMsg.setTask(input.getSourceTask());
+            boltMsg.setTuple(input.getValues());
+
+            _pendingWrites.put(boltMsg);
         } catch(InterruptedException e) {
             throw new RuntimeException("Error during multilang processing", e);
         }
@@ -184,9 +178,9 @@ public class ShellBolt implements IBolt {
         _collector.reportError(new Exception("Shell Process Exception: " + msg));
     }
 
-    private void handleEmit(Emission emission) throws InterruptedException {
+    private void handleEmit(ShellMsg shellMsg) throws InterruptedException {
     	List<Tuple> anchors = new ArrayList<Tuple>();
-    	for (String anchor : emission.getAnchors()) {
+    	for (String anchor : shellMsg.getAnchors()) {
 	    	Tuple t = _inputs.get(anchor);
 	        if (t == null) {
 	            throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
@@ -194,11 +188,12 @@ public class ShellBolt implements IBolt {
 	        anchors.add(t);
     	}
 
-        if(emission.getTask() == 0) {
-            List<Integer> outtasks = _collector.emit(emission.getStream(), anchors, emission.getTuple());
+        if(shellMsg.getTask() == 0) {
+            List<Integer> outtasks = _collector.emit(shellMsg.getStream(), anchors, shellMsg.getTuple());
             _pendingWrites.put(outtasks);
         } else {
-            _collector.emitDirect((int)emission.getTask(), emission.getStream(), anchors, emission.getTuple());
+            _collector.emitDirect((int) shellMsg.getTask(),
+                    shellMsg.getStream(), anchors, shellMsg.getTuple());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3219a4f0/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
index 38976bc..8871a77 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
@@ -1,9 +1,10 @@
 package backtype.storm.utils;
 
-import backtype.storm.multilang.Emission;
+import backtype.storm.Config;
 import backtype.storm.multilang.ISerializer;
-import backtype.storm.multilang.Immission;
+import backtype.storm.multilang.BoltMsg;
 import backtype.storm.multilang.NoOutputException;
+import backtype.storm.multilang.ShellMsg;
 import backtype.storm.multilang.SpoutMsg;
 import backtype.storm.task.TopologyContext;
 import java.io.File;
@@ -18,85 +19,109 @@ import org.apache.log4j.Logger;
 
 public class ShellProcess implements Serializable {
     public static Logger LOG = Logger.getLogger(ShellProcess.class);
-    private Process _subprocess;
-    private InputStream processErrorStream;
-    private String[] command;
-    public ISerializer serializer;
+    private Process      _subprocess;
+    private InputStream  processErrorStream;
+    private String[]     command;
+    public ISerializer   serializer;
 
-    public ShellProcess(ISerializer serializer, String[] command) {
+    public ShellProcess(String[] command) {
         this.command = command;
-        this.serializer = serializer;
     }
 
     public Number launch(Map conf, TopologyContext context) {
         ProcessBuilder builder = new ProcessBuilder(command);
         builder.directory(new File(context.getCodeDir()));
 
+        this.serializer = getSerializer(conf);
+
         Number pid;
         try {
-	        _subprocess = builder.start();
-	        processErrorStream = _subprocess.getErrorStream();
-	        serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream());
-	        pid = serializer.connect(conf, context);
+            _subprocess = builder.start();
+            processErrorStream = _subprocess.getErrorStream();
+            serializer.initialize(_subprocess.getOutputStream(),
+                    _subprocess.getInputStream());
+            pid = serializer.connect(conf, context);
         } catch (IOException e) {
-        	throw new RuntimeException("Error when launching multilang subprocess\n" + getErrorsString(), e);
+            throw new RuntimeException(
+                    "Error when launching multilang subprocess\n"
+                            + getErrorsString(), e);
         } catch (NoOutputException e) {
             throw new RuntimeException(e + getErrorsString() + "\n");
-    	}
+        }
         return pid;
     }
 
+    private ISerializer getSerializer(Map conf) {
+        //get factory class name
+        String serializer_className = (String)conf.get(Config.STORM_MULTILANG_SERIALIZER);
+        LOG.info("Storm multilang serializer:" + serializer_className);
+
+        ISerializer serializer = null;
+        try {
+            //create a factory class
+            Class klass = Class.forName(serializer_className);
+            //obtain a serializer object
+            Object obj = klass.newInstance();
+            serializer = (ISerializer)obj;
+        } catch(Exception e) {
+            throw new RuntimeException("Failed to construct multilang serializer from serializer " + serializer_className, e);
+        }
+        return serializer;
+    }
+
     public void destroy() {
         _subprocess.destroy();
     }
 
-    public Emission readEmission() throws IOException {
-    	try  {
-    		return serializer.readEmission();
-    	} catch (NoOutputException e) {
+    public ShellMsg readShellMsg() throws IOException {
+        try {
+            return serializer.readShellMsg();
+        } catch (NoOutputException e) {
             throw new RuntimeException(e + getErrorsString() + "\n");
-    	}
+        }
     }
 
-    public void writeImmission(Immission msg) throws IOException {
-    	serializer.writeImmission(msg);
-        // drain the error stream to avoid dead lock because of full error stream buffer
+    public void writeBoltMsg(BoltMsg msg) throws IOException {
+        serializer.writeBoltMsg(msg);
+        // drain the error stream to avoid dead lock because of full error
+        // stream buffer
         drainErrorStream();
     }
 
     public void writeSpoutMsg(SpoutMsg msg) throws IOException {
-    	serializer.writeSpoutMsg(msg);
-        // drain the error stream to avoid dead lock because of full error stream buffer
+        serializer.writeSpoutMsg(msg);
+        // drain the error stream to avoid dead lock because of full error
+        // stream buffer
         drainErrorStream();
     }
 
     public void writeTaskIds(List<Integer> taskIds) throws IOException {
-    	serializer.writeTaskIds(taskIds);
-        // drain the error stream to avoid dead lock because of full error stream buffer
+        serializer.writeTaskIds(taskIds);
+        // drain the error stream to avoid dead lock because of full error
+        // stream buffer
         drainErrorStream();
     }
 
-    public void drainErrorStream()
-    {
+    public void drainErrorStream() {
         try {
-            while (processErrorStream.available() > 0)
-            {
+            while (processErrorStream.available() > 0) {
                 int bufferSize = processErrorStream.available();
-                byte[] errorReadingBuffer =  new byte[bufferSize];
+                byte[] errorReadingBuffer = new byte[bufferSize];
 
                 processErrorStream.read(errorReadingBuffer, 0, bufferSize);
 
-                LOG.info("Got error from shell process: " + new String(errorReadingBuffer));
+                LOG.info("Got error from shell process: "
+                        + new String(errorReadingBuffer));
             }
-        } catch(Exception e) {
+        } catch (Exception e) {
         }
     }
 
     public String getErrorsString() {
-        if(processErrorStream!=null) {
+        if (processErrorStream != null) {
             try {
                 return IOUtils.toString(processErrorStream);
-            } catch(IOException e) {
+            } catch (IOException e) {
                 return "(Unable to capture error stream)";
             }
         } else {


[12/24] git commit: Removed pure whitespace changes of project.clj and bootstrap.clj

Posted by bo...@apache.org.
Removed pure whitespace changes of project.clj and bootstrap.clj


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

Branch: refs/heads/master
Commit: 16ba0c7fbba5b49ed4054f7aba7fc7e15e144c77
Parents: 5699373
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Thu Oct 10 14:28:08 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Thu Oct 10 14:28:08 2013 +0200

----------------------------------------------------------------------
 project.clj                                     | 2 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/16ba0c7f/project.clj
----------------------------------------------------------------------
diff --git a/project.clj b/project.clj
index 6e59efc..ed1fdbc 100644
--- a/project.clj
+++ b/project.clj
@@ -10,7 +10,7 @@
                  :archive "https://groups.google.com/group/storm-user"
                  :post "storm-user@googlegroups.com"}
   :dependencies [~@DEPENDENCIES]
-  :plugins [[~'lein-sub "0.2.1"]]
+  :plugins [[~'lein-sub "0.2.1"]]  
   :min-lein-version "2.0.0"
   :sub [~@MODULES]
   ))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/16ba0c7f/storm-core/src/clj/backtype/storm/bootstrap.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index 9edbe8e..70b8005 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -17,7 +17,7 @@
      (import (quote [backtype.storm.task IBolt IOutputCollector
                      OutputCollector TopologyContext ShellBolt
                      GeneralTopologyContext WorkerTopologyContext]))
-     (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs
+     (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs 
                      IBatchBolt BatchBoltExecutor]))
      (import (quote [backtype.storm.drpc KeyedFairBolt]))
      (import (quote [backtype.storm.daemon Shutdownable]))


[15/24] git commit: Merge branch 'master' of https://github.com/nathanmarz/storm

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/nathanmarz/storm


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

Branch: refs/heads/master
Commit: 80304231ff550f74b99a88684b23031208bec919
Parents: 84360dd 66a3973
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Mon Nov 18 16:30:44 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Mon Nov 18 16:30:44 2013 +0200

----------------------------------------------------------------------
 CHANGELOG.md                                    | 13 +++++-
 conf/defaults.yaml                              |  1 +
 .../src/clj/backtype/storm/daemon/logviewer.clj | 42 +++++++++++++++-----
 storm-core/src/clj/backtype/storm/ui/core.clj   |  6 +--
 storm-core/src/jvm/backtype/storm/Config.java   | 12 ++++++
 .../src/jvm/backtype/storm/task/ShellBolt.java  |  5 +++
 6 files changed, 65 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/80304231/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/80304231/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/80304231/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index 1649fbe,3fa6741..8efd904
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@@ -1,13 -1,14 +1,14 @@@
  package backtype.storm.task;
  
+ import backtype.storm.Config;
  import backtype.storm.generated.ShellComponent;
 -import backtype.storm.tuple.MessageId;
  import backtype.storm.tuple.Tuple;
 -import backtype.storm.utils.Utils;
  import backtype.storm.utils.ShellProcess;
 -import java.io.IOException;
 +import backtype.storm.multilang.BoltMsg;
 +import backtype.storm.multilang.ShellMsg;
 +
  import java.util.ArrayList;
 -import java.util.Arrays;
 +import java.util.List;
  import java.util.concurrent.ConcurrentHashMap;
  import java.util.concurrent.LinkedBlockingQueue;
  import static java.util.concurrent.TimeUnit.SECONDS;
@@@ -69,13 -72,21 +70,17 @@@ public class ShellBolt implements IBol
  
      public void prepare(Map stormConf, TopologyContext context,
                          final OutputCollector collector) {
+         Object maxPending = stormConf.get(Config.TOPOLOGY_SHELLBOLT_MAX_PENDING);
+         if (maxPending != null) {
+            this._pendingWrites = new LinkedBlockingQueue(((Number)maxPending).intValue());
+         }
          _rand = new Random();
 -        _process = new ShellProcess(_command);
          _collector = collector;
 +        _process = new ShellProcess(_command);
  
 -        try {
 -            //subprocesses must send their pid first thing
 -            Number subpid = _process.launch(stormConf, context);
 -            LOG.info("Launched subprocess with pid " + subpid);
 -        } catch (IOException e) {
 -            throw new RuntimeException("Error when launching multilang subprocess\n" + _process.getErrorsString(), e);
 -        }
 +        //subprocesses must send their pid first thing
 +        Number subpid = _process.launch(stormConf, context);
 +        LOG.info("Launched subprocess with pid " + subpid);
  
          // reader
          _readerThread = new Thread(new Runnable() {


[10/24] git commit: Added json msg and id parsing.

Posted by bo...@apache.org.
Added json msg and id parsing.


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

Branch: refs/heads/master
Commit: 32b371857ddec5669f65fa779f6a0b098c1cb0a1
Parents: 63afb94
Author: John Gilmore <jg...@ml.sun.ac.za>
Authored: Wed Oct 9 16:40:05 2013 +0200
Committer: John Gilmore <jg...@ml.sun.ac.za>
Committed: Wed Oct 9 16:40:05 2013 +0200

----------------------------------------------------------------------
 .../src/jvm/backtype/storm/multilang/JsonSerializer.java     | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/32b37185/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
index 8afa4bb..70d8e5b 100644
--- a/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/multilang/JsonSerializer.java
@@ -81,6 +81,12 @@ public class JsonSerializer implements ISerializer {
         String command = (String) msg.get("command");
         shellMsg.setCommand(command);
 
+        String id = (String) msg.get("id");
+        shellMsg.setId(id);
+
+        String log = (String) msg.get("msg");
+        shellMsg.setMsg(log);
+
         String stream = (String) msg.get("stream");
         if (stream == null)
             stream = Utils.DEFAULT_STREAM_ID;
@@ -92,7 +98,7 @@ public class JsonSerializer implements ISerializer {
         } else {
             shellMsg.setTask(0);
         }
-        
+
         Object need_task_ids = msg.get("need_task_ids");
         if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) {
             shellMsg.setNeedTaskIds(true);