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 2017/04/06 17:32:50 UTC

[01/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Repository: storm
Updated Branches:
  refs/heads/master c652d3ffc -> 4eb65075a


http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java
new file mode 100644
index 0000000..9f48c39
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.io.IOException;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SaslNettyServer {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(SaslNettyServer.class);
+
+        private SaslServer saslServer;
+
+    SaslNettyServer(String topologyName, byte[] token) throws IOException {
+        LOG.debug("SaslNettyServer: Topology token is: {} with authmethod {}",
+                  topologyName, SaslUtils.AUTH_DIGEST_MD5);
+
+        try {
+            SaslDigestCallbackHandler ch = new SaslNettyServer.SaslDigestCallbackHandler(
+                topologyName, token);
+
+            saslServer = Sasl.createSaslServer(SaslUtils.AUTH_DIGEST_MD5, null,
+                                               SaslUtils.DEFAULT_REALM,
+                                               SaslUtils.getSaslProps(), ch);
+        } catch (SaslException e) {
+            LOG.error("SaslNettyServer: Could not create SaslServer: ", e);
+        }
+    }
+
+    public boolean isComplete() {
+        return saslServer.isComplete();
+    }
+
+    public String getUserName() {
+        return saslServer.getAuthorizationID();
+    }
+
+    /** CallbackHandler for SASL DIGEST-MD5 mechanism */
+    public static class SaslDigestCallbackHandler implements CallbackHandler {
+
+        /** Used to authenticate the clients */
+        private byte[] userPassword;
+        private String userName;
+
+        public SaslDigestCallbackHandler(String topologyName, byte[] token) {
+            LOG.debug("SaslDigestCallback: Creating SaslDigestCallback handler with topology token: {}", topologyName);
+            this.userName = topologyName;
+            this.userPassword = token;
+        }
+
+        @Override
+        public void handle(Callback[] callbacks) throws IOException,
+            UnsupportedCallbackException {
+            NameCallback nc = null;
+            PasswordCallback pc = null;
+            AuthorizeCallback ac = null;
+
+            for (Callback callback : callbacks) {
+                if (callback instanceof AuthorizeCallback) {
+                    ac = (AuthorizeCallback) callback;
+                } else if (callback instanceof NameCallback) {
+                    nc = (NameCallback) callback;
+                } else if (callback instanceof PasswordCallback) {
+                    pc = (PasswordCallback) callback;
+                } else if (callback instanceof RealmCallback) {
+                    continue; // realm is ignored
+                } else {
+                    throw new UnsupportedCallbackException(callback,
+                                                           "handle: Unrecognized SASL DIGEST-MD5 Callback");
+                }
+            }
+
+            if (nc != null) {
+                LOG.debug("handle: SASL server DIGEST-MD5 callback: setting username for client: {}",
+                          userName);
+                nc.setName(userName);
+            }
+
+            if (pc != null) {
+                char[] password = SaslUtils.encodePassword(userPassword);
+
+                LOG.debug("handle: SASL server DIGEST-MD5 callback: setting password for client: ",
+                          userPassword);
+
+                pc.setPassword(password);
+            }
+            if (ac != null) {
+
+                String authid = ac.getAuthenticationID();
+                String authzid = ac.getAuthorizationID();
+
+                if (authid.equals(authzid)) {
+                    ac.setAuthorized(true);
+                } else {
+                    ac.setAuthorized(false);
+                }
+
+                if (ac.isAuthorized()) {
+                    LOG.debug("handle: SASL server DIGEST-MD5 callback: setting canonicalized client ID: ",
+                              userName);
+                    ac.setAuthorizedID(authzid);
+                }
+            }
+        }
+    }
+
+    /**
+     * Used by SaslTokenMessage::processToken() to respond to server SASL
+     * tokens.
+     *
+     * @param token
+     *            Server's SASL token
+     * @return token to send back to the server.
+     */
+    public byte[] response(byte[] token) {
+        try {
+            LOG.debug("response: Responding to input token of length: {}",
+                      token.length);
+            byte[] retval = saslServer.evaluateResponse(token);
+            LOG.debug("response: Response token length: {}", retval.length);
+            return retval;
+        } catch (SaslException e) {
+            LOG.error("response: Failed to evaluate client token of length: {} : {}",
+                      token.length, e);
+            return null;
+        }
+    }
+}


[39/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java b/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
new file mode 100644
index 0000000..5abb04a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import org.apache.storm.Config;
+import org.apache.storm.ILocalDRPC;
+import org.apache.storm.generated.DRPCRequest;
+import org.apache.storm.generated.DistributedRPCInvocations;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.ExtendedThreadPoolExecutor;
+import org.apache.storm.utils.ServiceRegistry;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.Callable;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.thrift.TException;
+import org.json.simple.JSONValue;
+
+public class DRPCSpout extends BaseRichSpout {
+    //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
+    static final long serialVersionUID = 2387848310969237877L;
+
+    public static final Logger LOG = LoggerFactory.getLogger(DRPCSpout.class);
+    
+    SpoutOutputCollector _collector;
+    List<DRPCInvocationsClient> _clients = new ArrayList<>();
+    transient LinkedList<Future<Void>> _futures = null;
+    transient ExecutorService _backround = null;
+    String _function;
+    String _local_drpc_id = null;
+    
+    private static class DRPCMessageId {
+        String id;
+        int index;
+        
+        public DRPCMessageId(String id, int index) {
+            this.id = id;
+            this.index = index;
+        }
+    }
+    
+    
+    public DRPCSpout(String function) {
+        _function = function;
+    }
+
+    public DRPCSpout(String function, ILocalDRPC drpc) {
+        _function = function;
+        _local_drpc_id = drpc.getServiceId();
+    }
+
+    public String get_function() {
+        return _function;
+    }
+
+    private class Adder implements Callable<Void> {
+        private String server;
+        private int port;
+        private Map conf;
+
+        public Adder(String server, int port, Map conf) {
+            this.server = server;
+            this.port = port;
+            this.conf = conf;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            DRPCInvocationsClient c = new DRPCInvocationsClient(conf, server, port);
+            synchronized (_clients) {
+                _clients.add(c);
+            }
+            return null;
+        }
+    }
+
+    private void reconnectAsync(final DRPCInvocationsClient client) {
+        _futures.add(_backround.submit(new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                client.reconnectClient();
+                return null;
+            }
+        }));
+    }
+
+    private void reconnectSync(DRPCInvocationsClient client) {
+        try {
+            LOG.info("reconnecting... ");
+            client.reconnectClient(); //Blocking call
+        } catch (TException e2) {
+            LOG.error("Failed to connect to DRPC server", e2);
+        }
+    }
+
+    private void checkFutures() {
+        Iterator<Future<Void>> i = _futures.iterator();
+        while (i.hasNext()) {
+            Future<Void> f = i.next();
+            if (f.isDone()) {
+                i.remove();
+            }
+            try {
+                f.get();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+ 
+    @Override
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        _collector = collector;
+        if(_local_drpc_id==null) {
+            _backround = new ExtendedThreadPoolExecutor(0, Integer.MAX_VALUE,
+                60L, TimeUnit.SECONDS,
+                new SynchronousQueue<Runnable>());
+            _futures = new LinkedList<>();
+
+            int numTasks = context.getComponentTasks(context.getThisComponentId()).size();
+            int index = context.getThisTaskIndex();
+
+            int port = ObjectReader.getInt(conf.get(Config.DRPC_INVOCATIONS_PORT));
+            List<String> servers = (List<String>) conf.get(Config.DRPC_SERVERS);
+            if(servers == null || servers.isEmpty()) {
+                throw new RuntimeException("No DRPC servers configured for topology");   
+            }
+            
+            if (numTasks < servers.size()) {
+                for (String s: servers) {
+                    _futures.add(_backround.submit(new Adder(s, port, conf)));
+                }
+            } else {        
+                int i = index % servers.size();
+                _futures.add(_backround.submit(new Adder(servers.get(i), port, conf)));
+            }
+        }
+        
+    }
+
+    @Override
+    public void close() {
+        for(DRPCInvocationsClient client: _clients) {
+            client.close();
+        }
+    }
+
+    @Override
+    public void nextTuple() {
+        boolean gotRequest = false;
+        if(_local_drpc_id==null) {
+            int size = 0;
+            synchronized (_clients) {
+                size = _clients.size(); //This will only ever grow, so no need to worry about falling off the end
+            }
+            for(int i=0; i<size; i++) {
+                DRPCInvocationsClient client;
+                synchronized (_clients) {
+                    client = _clients.get(i);
+                }
+                if (!client.isConnected()) {
+                    LOG.warn("DRPCInvocationsClient [{}:{}] is not connected.", client.getHost(), client.getPort());
+                    reconnectAsync(client);
+                    continue;
+                }
+                try {
+                    DRPCRequest req = client.fetchRequest(_function);
+                    if(req.get_request_id().length() > 0) {
+                        Map returnInfo = new HashMap();
+                        returnInfo.put("id", req.get_request_id());
+                        returnInfo.put("host", client.getHost());
+                        returnInfo.put("port", client.getPort());
+                        gotRequest = true;
+                        _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), new DRPCMessageId(req.get_request_id(), i));
+                        break;
+                    }
+                } catch (AuthorizationException aze) {
+                    reconnectAsync(client);
+                    LOG.error("Not authorized to fetch DRPC result from DRPC server", aze);
+                } catch (TException e) {
+                    reconnectAsync(client);
+                    LOG.error("Failed to fetch DRPC result from DRPC server", e);
+                } catch (Exception e) {
+                    LOG.error("Failed to fetch DRPC result from DRPC server", e);
+                }
+            }
+            checkFutures();
+        } else {
+            DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id);
+            if(drpc!=null) { // can happen during shutdown of drpc while topology is still up
+                try {
+                    DRPCRequest req = drpc.fetchRequest(_function);
+                    if(req.get_request_id().length() > 0) {
+                        Map returnInfo = new HashMap();
+                        returnInfo.put("id", req.get_request_id());
+                        returnInfo.put("host", _local_drpc_id);
+                        returnInfo.put("port", 0);
+                        gotRequest = true;
+                        _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), 
+                                        new DRPCMessageId(req.get_request_id(), 0));
+                    }
+                } catch (AuthorizationException aze) {
+                    throw new RuntimeException(aze);
+                } catch (TException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        }
+        if(!gotRequest) {
+            Utils.sleep(1);
+        }
+    }
+
+    @Override
+    public void ack(Object msgId) {
+    }
+
+    @Override
+    public void fail(Object msgId) {
+        DRPCMessageId did = (DRPCMessageId) msgId;
+        DistributedRPCInvocations.Iface client;
+        
+        if (_local_drpc_id == null) {
+            client = _clients.get(did.index);
+        } else {
+            client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id);
+        }
+
+        int retryCnt = 0;
+        int maxRetries = 3;
+
+        while (retryCnt < maxRetries) {
+            retryCnt++;
+            try {
+                client.failRequest(did.id);
+                break;
+            } catch (AuthorizationException aze) {
+                LOG.error("Not authorized to failRequest from DRPC server", aze);
+                throw new RuntimeException(aze);
+            } catch (TException tex) {
+                if (retryCnt >= maxRetries) {
+                    LOG.error("Failed to fail request", tex);
+                    break;
+                }
+                reconnectSync((DRPCInvocationsClient)client);
+            }
+        }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("args", "return-info"));
+    }    
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java b/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java
new file mode 100644
index 0000000..f57bbb1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JoinResult extends BaseRichBolt {
+    public static final Logger LOG = LoggerFactory.getLogger(JoinResult.class);
+
+    String returnComponent;
+    Map<Object, Tuple> returns = new HashMap<>();
+    Map<Object, Tuple> results = new HashMap<>();
+    OutputCollector _collector;
+
+    public JoinResult(String returnComponent) {
+        this.returnComponent = returnComponent;
+    }
+ 
+    public void prepare(Map map, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+    }
+
+    public void execute(Tuple tuple) {
+        Object requestId = tuple.getValue(0);
+        if(tuple.getSourceComponent().equals(returnComponent)) {
+            returns.put(requestId, tuple);
+        } else {
+            results.put(requestId, tuple);
+        }
+
+        if(returns.containsKey(requestId) && results.containsKey(requestId)) {
+            Tuple result = results.remove(requestId);
+            Tuple returner = returns.remove(requestId);
+            LOG.debug(result.getValue(1).toString());
+            List<Tuple> anchors = new ArrayList<>();
+            anchors.add(result);
+            anchors.add(returner);            
+            _collector.emit(anchors, new Values(""+result.getValue(1), returner.getValue(1)));
+            _collector.ack(result);
+            _collector.ack(returner);
+        }
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("result", "return-info"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java b/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java
new file mode 100644
index 0000000..39860e6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import org.apache.storm.coordination.CoordinatedBolt.FinishedCallback;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicBoltExecutor;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.KeyedRoundRobinQueue;
+import java.util.HashMap;
+import java.util.Map;
+
+
+public class KeyedFairBolt implements IRichBolt, FinishedCallback {
+    IRichBolt _delegate;
+    KeyedRoundRobinQueue<Tuple> _rrQueue;
+    Thread _executor;
+    FinishedCallback _callback;
+
+    public KeyedFairBolt(IRichBolt delegate) {
+        _delegate = delegate;
+    }
+    
+    public KeyedFairBolt(IBasicBolt delegate) {
+        this(new BasicBoltExecutor(delegate));
+    }
+    
+    
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        if(_delegate instanceof FinishedCallback) {
+            _callback = (FinishedCallback) _delegate;
+        }
+        _delegate.prepare(stormConf, context, collector);
+        _rrQueue = new KeyedRoundRobinQueue<Tuple>();
+        _executor = new Thread(new Runnable() {
+            public void run() {
+                try {
+                    while(true) {
+                        _delegate.execute(_rrQueue.take());
+                    }
+                } catch (InterruptedException e) {
+
+                }
+            }
+        });
+        _executor.setDaemon(true);
+        _executor.start();
+    }
+
+    public void execute(Tuple input) {
+        Object key = input.getValue(0);
+        _rrQueue.add(key, input);
+    }
+
+    public void cleanup() {
+        _executor.interrupt();
+        _delegate.cleanup();
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        _delegate.declareOutputFields(declarer);
+    }
+
+    public void finishedId(Object id) {
+        if(_callback!=null) {
+            _callback.finishedId(id);
+        }
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return new HashMap<String, Object>();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java
new file mode 100644
index 0000000..6f82f80
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import org.apache.storm.grouping.CustomStreamGrouping;
+import org.apache.storm.topology.ComponentConfigurationDeclarer;
+import org.apache.storm.tuple.Fields;
+
+public interface LinearDRPCInputDeclarer extends ComponentConfigurationDeclarer<LinearDRPCInputDeclarer> {
+    public LinearDRPCInputDeclarer fieldsGrouping(Fields fields);
+    public LinearDRPCInputDeclarer fieldsGrouping(String streamId, Fields fields);
+
+    public LinearDRPCInputDeclarer globalGrouping();
+    public LinearDRPCInputDeclarer globalGrouping(String streamId);
+
+    public LinearDRPCInputDeclarer shuffleGrouping();
+    public LinearDRPCInputDeclarer shuffleGrouping(String streamId);
+
+    public LinearDRPCInputDeclarer localOrShuffleGrouping();
+    public LinearDRPCInputDeclarer localOrShuffleGrouping(String streamId);
+    
+    public LinearDRPCInputDeclarer noneGrouping();
+    public LinearDRPCInputDeclarer noneGrouping(String streamId);
+
+    public LinearDRPCInputDeclarer allGrouping();
+    public LinearDRPCInputDeclarer allGrouping(String streamId);
+
+    public LinearDRPCInputDeclarer directGrouping();
+    public LinearDRPCInputDeclarer directGrouping(String streamId);
+
+    public LinearDRPCInputDeclarer partialKeyGrouping(Fields fields);
+    public LinearDRPCInputDeclarer partialKeyGrouping(String streamId, Fields fields);
+
+    public LinearDRPCInputDeclarer customGrouping(CustomStreamGrouping grouping);
+    public LinearDRPCInputDeclarer customGrouping(String streamId, CustomStreamGrouping grouping);
+    
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java
new file mode 100644
index 0000000..dc702a3
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java
@@ -0,0 +1,393 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import org.apache.storm.Constants;
+import org.apache.storm.ILocalDRPC;
+import org.apache.storm.coordination.BatchBoltExecutor;
+import org.apache.storm.coordination.CoordinatedBolt;
+import org.apache.storm.coordination.CoordinatedBolt.FinishedCallback;
+import org.apache.storm.coordination.CoordinatedBolt.IdStreamSpec;
+import org.apache.storm.coordination.CoordinatedBolt.SourceArgs;
+import org.apache.storm.coordination.IBatchBolt;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.grouping.CustomStreamGrouping;
+import org.apache.storm.grouping.PartialKeyGrouping;
+import org.apache.storm.topology.BaseConfigurationDeclarer;
+import org.apache.storm.topology.BasicBoltExecutor;
+import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.InputDeclarer;
+import org.apache.storm.topology.OutputFieldsGetter;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Fields;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+// Trident subsumes the functionality provided by this class, so it's deprecated
+@Deprecated
+public class LinearDRPCTopologyBuilder {    
+    String _function;
+    List<Component> _components = new ArrayList<Component>();
+    
+    
+    public LinearDRPCTopologyBuilder(String function) {
+        _function = function;
+    }
+        
+    public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt, Number parallelism) {
+        return addBolt(new BatchBoltExecutor(bolt), parallelism);
+    }
+    
+    public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt) {
+        return addBolt(bolt, 1);
+    }
+    
+    @Deprecated
+    public LinearDRPCInputDeclarer addBolt(IRichBolt bolt, Number parallelism) {
+        if(parallelism==null) parallelism = 1; 
+        Component component = new Component(bolt, parallelism.intValue());
+        _components.add(component);
+        return new InputDeclarerImpl(component);
+    }
+    
+    @Deprecated
+    public LinearDRPCInputDeclarer addBolt(IRichBolt bolt) {
+        return addBolt(bolt, null);
+    }
+    
+    public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt, Number parallelism) {
+        return addBolt(new BasicBoltExecutor(bolt), parallelism);
+    }
+
+    public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt) {
+        return addBolt(bolt, null);
+    }
+        
+    public StormTopology createLocalTopology(ILocalDRPC drpc) {
+        return createTopology(new DRPCSpout(_function, drpc));
+    }
+    
+    public StormTopology createRemoteTopology() {
+        return createTopology(new DRPCSpout(_function));
+    }
+    
+    
+    private StormTopology createTopology(DRPCSpout spout) {
+        final String SPOUT_ID = "spout";
+        final String PREPARE_ID = "prepare-request";
+        
+        TopologyBuilder builder = new TopologyBuilder();
+        builder.setSpout(SPOUT_ID, spout);
+        builder.setBolt(PREPARE_ID, new PrepareRequest())
+                .noneGrouping(SPOUT_ID);
+        int i=0;
+        for(; i<_components.size();i++) {
+            Component component = _components.get(i);
+            
+            Map<String, SourceArgs> source = new HashMap<String, SourceArgs>();
+            if (i==1) {
+                source.put(boltId(i-1), SourceArgs.single());
+            } else if (i>=2) {
+                source.put(boltId(i-1), SourceArgs.all());
+            }
+            IdStreamSpec idSpec = null;
+            if(i==_components.size()-1 && component.bolt instanceof FinishedCallback) {
+                idSpec = IdStreamSpec.makeDetectSpec(PREPARE_ID, PrepareRequest.ID_STREAM);
+            }
+            BoltDeclarer declarer = builder.setBolt(
+                    boltId(i),
+                    new CoordinatedBolt(component.bolt, source, idSpec),
+                    component.parallelism);
+            
+            for(Map<String, Object> conf: component.componentConfs) {
+                declarer.addConfigurations(conf);
+            }
+            
+            if(idSpec!=null) {
+                declarer.fieldsGrouping(idSpec.getGlobalStreamId().get_componentId(), PrepareRequest.ID_STREAM, new Fields("request"));
+            }
+            if(i==0 && component.declarations.isEmpty()) {
+                declarer.noneGrouping(PREPARE_ID, PrepareRequest.ARGS_STREAM);
+            } else {
+                String prevId;
+                if(i==0) {
+                    prevId = PREPARE_ID;
+                } else {
+                    prevId = boltId(i-1);
+                }
+                for(InputDeclaration declaration: component.declarations) {
+                    declaration.declare(prevId, declarer);
+                }
+            }
+            if(i>0) {
+                declarer.directGrouping(boltId(i-1), Constants.COORDINATED_STREAM_ID); 
+            }
+        }
+        
+        IRichBolt lastBolt = _components.get(_components.size()-1).bolt;
+        OutputFieldsGetter getter = new OutputFieldsGetter();
+        lastBolt.declareOutputFields(getter);
+        Map<String, StreamInfo> streams = getter.getFieldsDeclaration();
+        if(streams.size()!=1) {
+            throw new RuntimeException("Must declare exactly one stream from last bolt in LinearDRPCTopology");
+        }
+        String outputStream = streams.keySet().iterator().next();
+        List<String> fields = streams.get(outputStream).get_output_fields();
+        if(fields.size()!=2) {
+            throw new RuntimeException("Output stream of last component in LinearDRPCTopology must contain exactly two fields. The first should be the request id, and the second should be the result.");
+        }
+
+        builder.setBolt(boltId(i), new JoinResult(PREPARE_ID))
+                .fieldsGrouping(boltId(i-1), outputStream, new Fields(fields.get(0)))
+                .fieldsGrouping(PREPARE_ID, PrepareRequest.RETURN_STREAM, new Fields("request"));
+        i++;
+        builder.setBolt(boltId(i), new ReturnResults())
+                .noneGrouping(boltId(i-1));
+        return builder.createTopology();
+    }
+    
+    private static String boltId(int index) {
+        return "bolt" + index;
+    }
+    
+    private static class Component {
+        public IRichBolt bolt;
+        public int parallelism;
+        public List<Map<String, Object>> componentConfs = new ArrayList<>();
+        public List<InputDeclaration> declarations = new ArrayList<InputDeclaration>();
+        
+        public Component(IRichBolt bolt, int parallelism) {
+            this.bolt = bolt;
+            this.parallelism = parallelism;
+        }
+    }
+    
+    private static interface InputDeclaration {
+        public void declare(String prevComponent, InputDeclarer declarer);
+    }
+    
+    private static class InputDeclarerImpl extends BaseConfigurationDeclarer<LinearDRPCInputDeclarer> implements LinearDRPCInputDeclarer {
+        Component _component;
+        
+        public InputDeclarerImpl(Component component) {
+            _component = component;
+        }
+        
+        @Override
+        public LinearDRPCInputDeclarer fieldsGrouping(final Fields fields) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.fieldsGrouping(prevComponent, fields);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer fieldsGrouping(final String streamId, final Fields fields) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.fieldsGrouping(prevComponent, streamId, fields);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer globalGrouping() {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.globalGrouping(prevComponent);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer globalGrouping(final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.globalGrouping(prevComponent, streamId);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer shuffleGrouping() {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.shuffleGrouping(prevComponent);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer shuffleGrouping(final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.shuffleGrouping(prevComponent, streamId);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer localOrShuffleGrouping() {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.localOrShuffleGrouping(prevComponent);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer localOrShuffleGrouping(final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.localOrShuffleGrouping(prevComponent, streamId);
+                }                
+            });
+            return this;
+        }
+        
+        @Override
+        public LinearDRPCInputDeclarer noneGrouping() {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.noneGrouping(prevComponent);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer noneGrouping(final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.noneGrouping(prevComponent, streamId);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer allGrouping() {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.allGrouping(prevComponent);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer allGrouping(final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.allGrouping(prevComponent, streamId);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer directGrouping() {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.directGrouping(prevComponent);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer directGrouping(final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.directGrouping(prevComponent, streamId);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer partialKeyGrouping(Fields fields) {
+            return customGrouping(new PartialKeyGrouping(fields));
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer partialKeyGrouping(String streamId, Fields fields) {
+            return customGrouping(streamId, new PartialKeyGrouping(fields));
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer customGrouping(final CustomStreamGrouping grouping) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.customGrouping(prevComponent, grouping);
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer customGrouping(final String streamId, final CustomStreamGrouping grouping) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(String prevComponent, InputDeclarer declarer) {
+                    declarer.customGrouping(prevComponent, streamId, grouping);
+                }                
+            });
+            return this;
+        }
+        
+        private void addDeclaration(InputDeclaration declaration) {
+            _component.declarations.add(declaration);
+        }
+
+        @Override
+        public LinearDRPCInputDeclarer addConfigurations(Map<String, Object> conf) {
+            _component.componentConfs.add(conf);
+            return this;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/PrepareRequest.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/PrepareRequest.java b/storm-client/src/jvm/org/apache/storm/drpc/PrepareRequest.java
new file mode 100644
index 0000000..06e576c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/PrepareRequest.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.storm.utils.Utils;
+
+
+public class PrepareRequest extends BaseBasicBolt {
+    public static final String ARGS_STREAM = Utils.DEFAULT_STREAM_ID;
+    public static final String RETURN_STREAM = "ret";
+    public static final String ID_STREAM = "id";
+
+    Random rand;
+
+    @Override
+    public void prepare(Map map, TopologyContext context) {
+        rand = new Random();
+    }
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+        String args = tuple.getString(0);
+        String returnInfo = tuple.getString(1);
+        long requestId = rand.nextLong();
+        collector.emit(ARGS_STREAM, new Values(requestId, args));
+        collector.emit(RETURN_STREAM, new Values(requestId, returnInfo));
+        collector.emit(ID_STREAM, new Values(requestId));
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declareStream(ARGS_STREAM, new Fields("request", "args"));
+        declarer.declareStream(RETURN_STREAM, new Fields("request", "return"));
+        declarer.declareStream(ID_STREAM, new Fields("request"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java b/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java
new file mode 100644
index 0000000..8f03356
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.DistributedRPCInvocations;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.ServiceRegistry;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+import org.json.simple.JSONValue;
+import org.json.simple.parser.ParseException;
+
+
+public class ReturnResults extends BaseRichBolt {
+    //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
+    static final long serialVersionUID = -774882142710631591L;
+
+    public static final Logger LOG = LoggerFactory.getLogger(ReturnResults.class);
+    OutputCollector _collector;
+    boolean local;
+    Map _conf; 
+    Map<List, DRPCInvocationsClient> _clients = new HashMap<List, DRPCInvocationsClient>();
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _conf = stormConf;
+        _collector = collector;
+        local = stormConf.get(Config.STORM_CLUSTER_MODE).equals("local");
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        String result = (String) input.getValue(0);
+        String returnInfo = (String) input.getValue(1);
+        if (returnInfo!=null) {
+            Map retMap = null;
+            try {
+                retMap = (Map) JSONValue.parseWithException(returnInfo);
+            } catch (ParseException e) {
+                 LOG.error("Parseing returnInfo failed", e);
+                 _collector.fail(input);
+                 return;
+            }
+            final String host = (String) retMap.get("host");
+            final int port = ObjectReader.getInt(retMap.get("port"));
+            String id = (String) retMap.get("id");
+            DistributedRPCInvocations.Iface client;
+            if (local) {
+                client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host);
+            } else {
+                List server = new ArrayList() {{
+                    add(host);
+                    add(port);
+                }};
+            
+                if(!_clients.containsKey(server)) {
+                    try {
+                        _clients.put(server, new DRPCInvocationsClient(_conf, host, port));
+                    } catch (TTransportException ex) {
+                        throw new RuntimeException(ex);
+                    }
+                }
+                client = _clients.get(server);
+            }
+ 
+
+            int retryCnt = 0;
+            int maxRetries = 3;
+            while (retryCnt < maxRetries) {
+                retryCnt++;
+                try {
+                    client.result(id, result);
+                    _collector.ack(input);
+                    break;
+                } catch (AuthorizationException aze) {
+                    LOG.error("Not authorized to return results to DRPC server", aze);
+                    _collector.fail(input);
+                    throw new RuntimeException(aze);
+                } catch (TException tex) {
+                    if (retryCnt >= maxRetries) {
+                        LOG.error("Failed to return results to DRPC server", tex);
+                        _collector.fail(input);
+                    }
+                    reconnectClient((DRPCInvocationsClient) client);
+                }
+            }
+        }
+    }    
+
+    private void reconnectClient(DRPCInvocationsClient client) {
+        if (client instanceof DRPCInvocationsClient) {
+            try {
+                LOG.info("reconnecting... ");
+                client.reconnectClient(); //Blocking call
+            } catch (TException e2) {
+                LOG.error("Failed to connect to DRPC server", e2);
+            }
+        }
+    }
+    @Override
+    public void cleanup() {
+        for(DRPCInvocationsClient c: _clients.values()) {
+            c.close();
+        }
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/Executor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/Executor.java b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
new file mode 100644
index 0000000..d89ac69
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
@@ -0,0 +1,583 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.json.simple.parser.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler<Object> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Executor.class);
+
+    protected final WorkerState workerData;
+    protected final WorkerTopologyContext workerTopologyContext;
+    protected final List<Long> executorId;
+    protected final List<Integer> taskIds;
+    protected final String componentId;
+    protected final AtomicBoolean openOrPrepareWasCalled;
+    protected final Map stormConf;
+    protected final Map conf;
+    protected final String stormId;
+    protected final HashMap sharedExecutorData;
+    protected final AtomicBoolean stormActive;
+    protected final AtomicReference<Map<String, DebugOptions>> stormComponentDebug;
+    protected final Runnable suicideFn;
+    protected final IStormClusterState stormClusterState;
+    protected final Map<Integer, String> taskToComponent;
+    protected CommonStats stats;
+    protected final Map<Integer, Map<Integer, Map<String, IMetric>>> intervalToTaskToMetricToRegistry;
+    protected final Map<String, Map<String, LoadAwareCustomStreamGrouping>> streamToComponentToGrouper;
+    protected final ReportErrorAndDie reportErrorDie;
+    protected final Callable<Boolean> sampler;
+    protected ExecutorTransfer executorTransfer;
+    protected final String type;
+    protected final AtomicBoolean throttleOn;
+
+    protected final IReportError reportError;
+    protected final Random rand;
+    protected final DisruptorQueue transferQueue;
+    protected final DisruptorQueue receiveQueue;
+    protected Map<Integer, Task> idToTask;
+    protected final Map<String, String> credentials;
+    protected final Boolean isDebug;
+    protected final Boolean hasEventLoggers;
+    protected String hostname;
+
+    protected Executor(WorkerState workerData, List<Long> executorId, Map<String, String> credentials) {
+        this.workerData = workerData;
+        this.executorId = executorId;
+        this.workerTopologyContext = workerData.getWorkerTopologyContext();
+        this.taskIds = StormCommon.executorIdToTasks(executorId);
+        this.componentId = workerTopologyContext.getComponentId(taskIds.get(0));
+        this.openOrPrepareWasCalled = new AtomicBoolean(false);
+        this.stormConf = normalizedComponentConf(workerData.getTopologyConf(), workerTopologyContext, componentId);
+        this.receiveQueue = (workerData.getExecutorReceiveQueueMap().get(executorId));
+        this.stormId = workerData.getTopologyId();
+        this.conf = workerData.getConf();
+        this.sharedExecutorData = new HashMap();
+        this.stormActive = workerData.getIsTopologyActive();
+        this.stormComponentDebug = workerData.getStormComponentToDebug();
+
+        this.transferQueue = mkExecutorBatchQueue(stormConf, executorId);
+        this.executorTransfer = new ExecutorTransfer(workerData, transferQueue, stormConf);
+
+        this.suicideFn = workerData.getSuicideCallback();
+        try {
+            this.stormClusterState = ClusterUtils.mkStormClusterState(workerData.getStateStorage(), Utils.getWorkerACL(stormConf),
+                    new ClusterStateContext(DaemonType.WORKER));
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+
+        StormTopology topology = workerTopologyContext.getRawTopology();
+        Map<String, SpoutSpec> spouts = topology.get_spouts();
+        Map<String, Bolt> bolts = topology.get_bolts();
+        if (spouts.containsKey(componentId)) {
+            this.type = StatsUtil.SPOUT;
+            this.stats = new SpoutExecutorStats(ConfigUtils.samplingRate(stormConf),ObjectReader.getInt(stormConf.get(Config.NUM_STAT_BUCKETS)));
+        } else if (bolts.containsKey(componentId)) {
+            this.type = StatsUtil.BOLT;
+            this.stats = new BoltExecutorStats(ConfigUtils.samplingRate(stormConf),ObjectReader.getInt(stormConf.get(Config.NUM_STAT_BUCKETS)));
+        } else {
+            throw new RuntimeException("Could not find " + componentId + " in " + topology);
+        }
+
+        this.intervalToTaskToMetricToRegistry = new HashMap<>();
+        this.taskToComponent = workerData.getTaskToComponent();
+        this.streamToComponentToGrouper = outboundComponents(workerTopologyContext, componentId, stormConf);
+        this.reportError = new ReportError(stormConf, stormClusterState, stormId, componentId, workerTopologyContext);
+        this.reportErrorDie = new ReportErrorAndDie(reportError, suicideFn);
+        this.sampler = ConfigUtils.mkStatsSampler(stormConf);
+        this.throttleOn = workerData.getThrottleOn();
+        this.isDebug = ObjectReader.getBoolean(stormConf.get(Config.TOPOLOGY_DEBUG), false);
+        this.rand = new Random(Utils.secureRandomLong());
+        this.credentials = credentials;
+        this.hasEventLoggers = StormCommon.hasEventLoggers(stormConf);
+
+        try {
+            this.hostname = Utils.hostname();
+        } catch (UnknownHostException ignored) {
+            this.hostname = "";
+        }
+    }
+
+    public static Executor mkExecutor(WorkerState workerState, List<Long> executorId, Map<String, String> credentials) {
+        Executor executor;
+
+        WorkerTopologyContext workerTopologyContext = workerState.getWorkerTopologyContext();
+        List<Integer> taskIds = StormCommon.executorIdToTasks(executorId);
+        String componentId = workerTopologyContext.getComponentId(taskIds.get(0));
+
+        String type = getExecutorType(workerTopologyContext, componentId);
+        if (StatsUtil.SPOUT.equals(type)) {
+            executor = new SpoutExecutor(workerState, executorId, credentials);
+            executor.stats = new SpoutExecutorStats(ConfigUtils.samplingRate(executor.getStormConf()),ObjectReader.getInt(executor.getStormConf().get(Config.NUM_STAT_BUCKETS)));
+        } else {
+            executor = new BoltExecutor(workerState, executorId, credentials);
+            executor.stats = new BoltExecutorStats(ConfigUtils.samplingRate(executor.getStormConf()),ObjectReader.getInt(executor.getStormConf().get(Config.NUM_STAT_BUCKETS)));
+        }
+
+        Map<Integer, Task> idToTask = new HashMap<>();
+        for (Integer taskId : taskIds) {
+            try {
+                Task task = new Task(executor, taskId);
+                executor.sendUnanchored(
+                        task, StormCommon.SYSTEM_STREAM_ID, new Values("startup"), executor.getExecutorTransfer());
+                idToTask.put(taskId, task);
+            } catch (IOException ex) {
+                throw Utils.wrapInRuntime(ex);
+            }
+        }
+
+        executor.idToTask = idToTask;
+        return executor;
+    }
+
+    private static String getExecutorType(WorkerTopologyContext workerTopologyContext, String componentId) {
+        StormTopology topology = workerTopologyContext.getRawTopology();
+        Map<String, SpoutSpec> spouts = topology.get_spouts();
+        Map<String, Bolt> bolts = topology.get_bolts();
+        if (spouts.containsKey(componentId)) {
+            return StatsUtil.SPOUT;
+        } else if (bolts.containsKey(componentId)) {
+            return StatsUtil.BOLT;
+        } else {
+            throw new RuntimeException("Could not find " + componentId + " in " + topology);
+        }
+    }
+
+    /**
+     * separated from mkExecutor in order to replace executor transfer in executor data for testing
+     */
+    public ExecutorShutdown execute() throws Exception {
+        LOG.info("Loading executor tasks " + componentId + ":" + executorId);
+
+        registerBackpressure();
+        Utils.SmartThread systemThreads =
+                Utils.asyncLoop(executorTransfer, executorTransfer.getName(), reportErrorDie);
+
+        String handlerName = componentId + "-executor" + executorId;
+        Utils.SmartThread handlers =
+                Utils.asyncLoop(this, false, reportErrorDie, Thread.NORM_PRIORITY, true, true, handlerName);
+        setupTicks(StatsUtil.SPOUT.equals(type));
+
+        LOG.info("Finished loading executor " + componentId + ":" + executorId);
+        return new ExecutorShutdown(this, Lists.newArrayList(systemThreads, handlers), idToTask);
+    }
+
+    public abstract void tupleActionFn(int taskId, TupleImpl tuple) throws Exception;
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void onEvent(Object event, long seq, boolean endOfBatch) throws Exception {
+        ArrayList<AddressedTuple> addressedTuples = (ArrayList<AddressedTuple>) event;
+        for (AddressedTuple addressedTuple : addressedTuples) {
+            TupleImpl tuple = (TupleImpl) addressedTuple.getTuple();
+            int taskId = addressedTuple.getDest();
+            if (isDebug) {
+                LOG.info("Processing received message FOR {} TUPLE: {}", taskId, tuple);
+            }
+            if (taskId != AddressedTuple.BROADCAST_DEST) {
+                tupleActionFn(taskId, tuple);
+            } else {
+                for (Integer t : taskIds) {
+                    tupleActionFn(t, tuple);
+                }
+            }
+        }
+    }
+
+    public void metricsTick(Task taskData, TupleImpl tuple) {
+        try {
+            Integer interval = tuple.getInteger(0);
+            int taskId = taskData.getTaskId();
+            Map<Integer, Map<String, IMetric>> taskToMetricToRegistry = intervalToTaskToMetricToRegistry.get(interval);
+            Map<String, IMetric> nameToRegistry = null;
+            if (taskToMetricToRegistry != null) {
+                nameToRegistry = taskToMetricToRegistry.get(taskId);
+            }
+            if (nameToRegistry != null) {
+                IMetricsConsumer.TaskInfo taskInfo = new IMetricsConsumer.TaskInfo(
+                        hostname, workerTopologyContext.getThisWorkerPort(),
+                        componentId, taskId, Time.currentTimeSecs(), interval);
+                List<IMetricsConsumer.DataPoint> dataPoints = new ArrayList<>();
+                for (Map.Entry<String, IMetric> entry : nameToRegistry.entrySet()) {
+                    IMetric metric = entry.getValue();
+                    Object value = metric.getValueAndReset();
+                    if (value != null) {
+                        IMetricsConsumer.DataPoint dataPoint = new IMetricsConsumer.DataPoint(entry.getKey(), value);
+                        dataPoints.add(dataPoint);
+                    }
+                }
+                if (!dataPoints.isEmpty()) {
+                    sendUnanchored(taskData, Constants.METRICS_STREAM_ID,
+                            new Values(taskInfo, dataPoints), executorTransfer);
+                }
+            }
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    protected void setupMetrics() {
+        for (final Integer interval : intervalToTaskToMetricToRegistry.keySet()) {
+            StormTimer timerTask = workerData.getUserTimer();
+            timerTask.scheduleRecurring(interval, interval, new Runnable() {
+                @Override
+                public void run() {
+                    TupleImpl tuple = new TupleImpl(workerTopologyContext, new Values(interval),
+                            (int) Constants.SYSTEM_TASK_ID, Constants.METRICS_TICK_STREAM_ID);
+                    List<AddressedTuple> metricsTickTuple =
+                            Lists.newArrayList(new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple));
+                    receiveQueue.publish(metricsTickTuple);
+                }
+            });
+        }
+    }
+
+    public void sendUnanchored(Task task, String stream, List<Object> values, ExecutorTransfer transfer) {
+        Tuple tuple = task.getTuple(stream, values);
+        List<Integer> tasks = task.getOutgoingTasks(stream, values);
+        for (Integer t : tasks) {
+            transfer.transfer(t, tuple);
+        }
+    }
+
+    /**
+     * Send sampled data to the eventlogger if the global or component level debug flag is set (via nimbus api).
+     */
+    public void sendToEventLogger(Executor executor, Task taskData, List values,
+                                  String componentId, Object messageId, Random random) {
+        Map<String, DebugOptions> componentDebug = executor.getStormComponentDebug().get();
+        DebugOptions debugOptions = componentDebug.get(componentId);
+        if (debugOptions == null) {
+            debugOptions = componentDebug.get(executor.getStormId());
+        }
+        double spct = ((debugOptions != null) && (debugOptions.is_enable())) ? debugOptions.get_samplingpct() : 0;
+        if (spct > 0 && (random.nextDouble() * 100) < spct) {
+            sendUnanchored(taskData, StormCommon.EVENTLOGGER_STREAM_ID,
+                    new Values(componentId, messageId, System.currentTimeMillis(), values),
+                    executor.getExecutorTransfer());
+        }
+    }
+
+    private void registerBackpressure() {
+        receiveQueue.registerBackpressureCallback(new DisruptorBackpressureCallback() {
+            @Override
+            public void highWaterMark() throws Exception {
+                LOG.debug("executor " + executorId + " is congested, set backpressure flag true");
+                WorkerBackpressureThread.notifyBackpressureChecker(workerData.getBackpressureTrigger());
+            }
+
+            @Override
+            public void lowWaterMark() throws Exception {
+                LOG.debug("executor " + executorId + " is not-congested, set backpressure flag false");
+                WorkerBackpressureThread.notifyBackpressureChecker(workerData.getBackpressureTrigger());
+            }
+        });
+        receiveQueue.setHighWaterMark(ObjectReader.getDouble(stormConf.get(Config.BACKPRESSURE_DISRUPTOR_HIGH_WATERMARK)));
+        receiveQueue.setLowWaterMark(ObjectReader.getDouble(stormConf.get(Config.BACKPRESSURE_DISRUPTOR_LOW_WATERMARK)));
+        receiveQueue.setEnableBackpressure(ObjectReader.getBoolean(stormConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE), false));
+    }
+
+    protected void setupTicks(boolean isSpout) {
+        final Integer tickTimeSecs = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS), null);
+        boolean enableMessageTimeout = (Boolean) stormConf.get(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS);
+        if (tickTimeSecs != null) {
+            if (Utils.isSystemId(componentId) || (!enableMessageTimeout && isSpout)) {
+                LOG.info("Timeouts disabled for executor " + componentId + ":" + executorId);
+            } else {
+                StormTimer timerTask = workerData.getUserTimer();
+                timerTask.scheduleRecurring(tickTimeSecs, tickTimeSecs, new Runnable() {
+                    @Override
+                    public void run() {
+                        TupleImpl tuple = new TupleImpl(workerTopologyContext, new Values(tickTimeSecs),
+                                (int) Constants.SYSTEM_TASK_ID, Constants.SYSTEM_TICK_STREAM_ID);
+                        List<AddressedTuple> tickTuple =
+                                Lists.newArrayList(new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple));
+                        receiveQueue.publish(tickTuple);
+                    }
+                });
+            }
+        }
+    }
+
+
+    private DisruptorQueue mkExecutorBatchQueue(Map stormConf, List<Long> executorId) {
+        int sendSize = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE));
+        int waitTimeOutMs = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS));
+        int batchSize = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_DISRUPTOR_BATCH_SIZE));
+        int batchTimeOutMs = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_DISRUPTOR_BATCH_TIMEOUT_MILLIS));
+        return new DisruptorQueue("executor" + executorId + "-send-queue", ProducerType.SINGLE,
+                sendSize, waitTimeOutMs, batchSize, batchTimeOutMs);
+    }
+
+    /**
+     * Returns map of stream id to component id to grouper
+     */
+    private Map<String, Map<String, LoadAwareCustomStreamGrouping>> outboundComponents(
+            WorkerTopologyContext workerTopologyContext, String componentId, Map stormConf) {
+        Map<String, Map<String, LoadAwareCustomStreamGrouping>> ret = new HashMap<>();
+
+        Map<String, Map<String, Grouping>> outputGroupings = workerTopologyContext.getTargets(componentId);
+        for (Map.Entry<String, Map<String, Grouping>> entry : outputGroupings.entrySet()) {
+            String streamId = entry.getKey();
+            Map<String, Grouping> componentGrouping = entry.getValue();
+            Fields outFields = workerTopologyContext.getComponentOutputFields(componentId, streamId);
+            Map<String, LoadAwareCustomStreamGrouping> componentGrouper = new HashMap<String, LoadAwareCustomStreamGrouping>();
+            for (Map.Entry<String, Grouping> cg : componentGrouping.entrySet()) {
+                String component = cg.getKey();
+                Grouping grouping = cg.getValue();
+                List<Integer> outTasks = workerTopologyContext.getComponentTasks(component);
+                LoadAwareCustomStreamGrouping grouper = GrouperFactory.mkGrouper(
+                        workerTopologyContext, componentId, streamId, outFields, grouping, outTasks, stormConf);
+                componentGrouper.put(component, grouper);
+            }
+            if (componentGrouper.size() > 0) {
+                ret.put(streamId, componentGrouper);
+            }
+        }
+
+        for (String stream : workerTopologyContext.getComponentCommon(componentId).get_streams().keySet()) {
+            if (!ret.containsKey(stream)) {
+                ret.put(stream, null);
+            }
+        }
+
+        return ret;
+    }
+
+    private Map normalizedComponentConf(Map stormConf, WorkerTopologyContext topologyContext, String componentId) {
+        List<Object> keysToRemove = All_CONFIGS();
+        keysToRemove.remove(Config.TOPOLOGY_DEBUG);
+        keysToRemove.remove(Config.TOPOLOGY_MAX_SPOUT_PENDING);
+        keysToRemove.remove(Config.TOPOLOGY_MAX_TASK_PARALLELISM);
+        keysToRemove.remove(Config.TOPOLOGY_TRANSACTIONAL_ID);
+        keysToRemove.remove(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS);
+        keysToRemove.remove(Config.TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS);
+        keysToRemove.remove(Config.TOPOLOGY_SPOUT_WAIT_STRATEGY);
+        keysToRemove.remove(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT);
+        keysToRemove.remove(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS);
+        keysToRemove.remove(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT);
+        keysToRemove.remove(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS);
+        keysToRemove.remove(Config.TOPOLOGY_BOLTS_TUPLE_TIMESTAMP_MAX_LAG_MS);
+        keysToRemove.remove(Config.TOPOLOGY_BOLTS_MESSAGE_ID_FIELD_NAME);
+        keysToRemove.remove(Config.TOPOLOGY_STATE_PROVIDER);
+        keysToRemove.remove(Config.TOPOLOGY_STATE_PROVIDER_CONFIG);
+        keysToRemove.remove(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM);
+
+        Map<Object, Object> componentConf;
+        String specJsonConf = topologyContext.getComponentCommon(componentId).get_json_conf();
+        if (specJsonConf != null) {
+            try {
+                componentConf = (Map<Object, Object>) JSONValue.parseWithException(specJsonConf);
+            } catch (ParseException e) {
+                throw new RuntimeException(e);
+            }
+            for (Object p : keysToRemove) {
+                componentConf.remove(p);
+            }
+        } else {
+            componentConf = new HashMap<>();
+        }
+
+        Map<Object, Object> ret = new HashMap<>();
+        ret.putAll(stormConf);
+        ret.putAll(componentConf);
+
+        return ret;
+    }
+
+    // =============================================================================
+    // ============================ getter methods =================================
+    // =============================================================================
+
+    public List<Long> getExecutorId() {
+        return executorId;
+    }
+
+    public List<Integer> getTaskIds() {
+        return taskIds;
+    }
+
+    public String getComponentId() {
+        return componentId;
+    }
+
+    public AtomicBoolean getOpenOrPrepareWasCalled() {
+        return openOrPrepareWasCalled;
+    }
+
+    public Map getStormConf() {
+        return stormConf;
+    }
+
+    public String getStormId() {
+        return stormId;
+    }
+
+    public CommonStats getStats() {
+        return stats;
+    }
+
+    public AtomicBoolean getThrottleOn() {
+        return throttleOn;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public Boolean getIsDebug() {
+        return isDebug;
+    }
+
+    public ExecutorTransfer getExecutorTransfer() {
+        return executorTransfer;
+    }
+
+    public IReportError getReportError() {
+        return reportError;
+    }
+
+    public WorkerTopologyContext getWorkerTopologyContext() {
+        return workerTopologyContext;
+    }
+
+    public Callable<Boolean> getSampler() {
+        return sampler;
+    }
+
+    public AtomicReference<Map<String, DebugOptions>> getStormComponentDebug() {
+        return stormComponentDebug;
+    }
+
+    public DisruptorQueue getReceiveQueue() {
+        return receiveQueue;
+    }
+
+    public boolean getBackpressure() {
+        return receiveQueue.getThrottleOn();
+    }
+
+    public DisruptorQueue getTransferWorkerQueue() {
+        return transferQueue;
+    }
+
+    public IStormClusterState getStormClusterState() {
+        return stormClusterState;
+    }
+
+    public WorkerState getWorkerData() {
+        return workerData;
+    }
+
+    public Map<String, Map<String, LoadAwareCustomStreamGrouping>> getStreamToComponentToGrouper() {
+        return streamToComponentToGrouper;
+    }
+
+    public HashMap getSharedExecutorData() {
+        return sharedExecutorData;
+    }
+
+    public Map<Integer, Map<Integer, Map<String, IMetric>>> getIntervalToTaskToMetricToRegistry() {
+        return intervalToTaskToMetricToRegistry;
+    }
+
+    @VisibleForTesting
+    public void setLocalExecutorTransfer(ExecutorTransfer executorTransfer) {
+        this.executorTransfer = executorTransfer;
+    }
+
+    private static List<Object> All_CONFIGS() {
+        List<Object> ret = new ArrayList<Object>();
+        Config config = new Config();
+        Class<?> ConfigClass = config.getClass();
+        Field[] fields = ConfigClass.getFields();
+        for (int i = 0; i < fields.length; i++) {
+            try {
+                Object obj = fields[i].get(null);
+                ret.add(obj);
+            } catch (IllegalArgumentException e) {
+                LOG.error(e.getMessage(), e);
+            } catch (IllegalAccessException e) {
+                LOG.error(e.getMessage(), e);
+            }
+        }
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/ExecutorShutdown.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/ExecutorShutdown.java b/storm-client/src/jvm/org/apache/storm/executor/ExecutorShutdown.java
new file mode 100644
index 0000000..144ee1b
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/ExecutorShutdown.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor;
+
+import com.google.common.collect.Lists;
+import org.apache.storm.Constants;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.generated.Credentials;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.spout.ISpout;
+import org.apache.storm.task.IBolt;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+public class ExecutorShutdown implements Shutdownable, IRunningExecutor {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ExecutorShutdown.class);
+    private final Executor executor;
+    private final List<Utils.SmartThread> threads;
+    private final Map<Integer, Task> taskDatas;
+
+    public ExecutorShutdown(Executor executor, List<Utils.SmartThread> threads, Map<Integer, Task> taskDatas) {
+        this.executor = executor;
+        this.threads = threads;
+        this.taskDatas = taskDatas;
+    }
+
+    @Override
+    public ExecutorStats renderStats() {
+        return executor.getStats().renderStats();
+    }
+
+    @Override
+    public List<Long> getExecutorId() {
+        return executor.getExecutorId();
+    }
+
+    @Override
+    public void credentialsChanged(Credentials credentials) {
+        TupleImpl tuple = new TupleImpl(executor.getWorkerTopologyContext(), new Values(credentials), (int) Constants.SYSTEM_TASK_ID,
+                Constants.CREDENTIALS_CHANGED_STREAM_ID);
+        List<AddressedTuple> addressedTuple = Lists.newArrayList(new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple));
+        executor.getReceiveQueue().publish(addressedTuple);
+    }
+
+    @Override
+    public boolean getBackPressureFlag() {
+        return executor.getBackpressure();
+    }
+
+    @Override
+    public void shutdown() {
+        try {
+            LOG.info("Shutting down executor " + executor.getComponentId() + ":" + executor.getExecutorId());
+            executor.getReceiveQueue().haltWithInterrupt();
+            executor.getTransferWorkerQueue().haltWithInterrupt();
+            for (Utils.SmartThread t : threads) {
+                t.interrupt();
+            }
+            for (Utils.SmartThread t : threads) {
+                LOG.debug("Executor " + executor.getComponentId() + ":" + executor.getExecutorId() + " joining thread " + t.getName());
+                t.join();
+            }
+            executor.getStats().cleanupStats();
+            for (Task task : taskDatas.values()) {
+                TopologyContext userContext = task.getUserContext();
+                for (ITaskHook hook : userContext.getHooks()) {
+                    hook.cleanup();
+                }
+            }
+            executor.getStormClusterState().disconnect();
+            if (executor.getOpenOrPrepareWasCalled().get()) {
+                for (Task task : taskDatas.values()) {
+                    Object object = task.getTaskObject();
+                    if (object instanceof ISpout) {
+                        ((ISpout) object).close();
+                    } else if (object instanceof IBolt) {
+                        ((IBolt) object).cleanup();
+                    } else {
+                        LOG.error("unknown component object");
+                    }
+                }
+            }
+            LOG.info("Shut down executor " + executor.getComponentId() + ":" + executor.getExecutorId());
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java b/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java
new file mode 100644
index 0000000..0012e4d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.lmax.disruptor.EventHandler;
+import org.apache.storm.Config;
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.serialization.KryoTupleSerializer;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.MutableObject;
+import org.apache.storm.utils.ObjectReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+public class ExecutorTransfer implements EventHandler, Callable {
+    private static final Logger LOG = LoggerFactory.getLogger(ExecutorTransfer.class);
+
+    private final WorkerState workerData;
+    private final DisruptorQueue batchTransferQueue;
+    private final Map stormConf;
+    private final KryoTupleSerializer serializer;
+    private final MutableObject cachedEmit;
+    private final boolean isDebug;
+
+    public ExecutorTransfer(WorkerState workerData, DisruptorQueue batchTransferQueue, Map stormConf) {
+        this.workerData = workerData;
+        this.batchTransferQueue = batchTransferQueue;
+        this.stormConf = stormConf;
+        this.serializer = new KryoTupleSerializer(stormConf, workerData.getWorkerTopologyContext());
+        this.cachedEmit = new MutableObject(new ArrayList<>());
+        this.isDebug = ObjectReader.getBoolean(stormConf.get(Config.TOPOLOGY_DEBUG), false);
+    }
+
+    public void transfer(int task, Tuple tuple) {
+        AddressedTuple val = new AddressedTuple(task, tuple);
+        if (isDebug) {
+            LOG.info("TRANSFERRING tuple {}", val);
+        }
+        batchTransferQueue.publish(val);
+    }
+
+    @VisibleForTesting
+    public DisruptorQueue getBatchTransferQueue() {
+        return this.batchTransferQueue;
+    }
+
+    @Override
+    public Object call() throws Exception {
+        batchTransferQueue.consumeBatchWhenAvailable(this);
+        return 0L;
+    }
+
+    public String getName() {
+        return batchTransferQueue.getName();
+    }
+
+    @Override
+    public void onEvent(Object event, long sequence, boolean endOfBatch) throws Exception {
+        ArrayList cachedEvents = (ArrayList) cachedEmit.getObject();
+        cachedEvents.add(event);
+        if (endOfBatch) {
+            workerData.transfer(serializer, cachedEvents);
+            cachedEmit.setObject(new ArrayList<>());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/IRunningExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/IRunningExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/IRunningExecutor.java
new file mode 100644
index 0000000..441fdff
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/IRunningExecutor.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor;
+
+import org.apache.storm.generated.Credentials;
+import org.apache.storm.generated.ExecutorStats;
+
+import java.util.List;
+
+public interface IRunningExecutor {
+
+    ExecutorStats renderStats();
+    List<Long> getExecutorId();
+    void credentialsChanged(Credentials credentials);
+    boolean getBackPressureFlag();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/LocalExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/LocalExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/LocalExecutor.java
new file mode 100644
index 0000000..4f2811b
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/LocalExecutor.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.executor;
+
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.RegisteredGlobalState;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class LocalExecutor {
+
+    private static volatile String trackId = null;
+
+    public static Executor mkExecutor(WorkerState workerState, List<Long> executorId, Map<String, String> initialCredentials)
+        throws Exception {
+        Executor executor = Executor.mkExecutor(workerState, executorId, initialCredentials);
+        executor.setLocalExecutorTransfer(new ExecutorTransfer(workerState, executor.getTransferWorkerQueue(),
+            executor.getStormConf()) {
+            @Override
+            public void transfer(int task, Tuple tuple) {
+                if (null != trackId) {
+                    ((AtomicInteger) ((Map) RegisteredGlobalState.getState(trackId)).get("transferred")).incrementAndGet();
+                }
+                super.transfer(task, tuple);
+            }
+        });
+        return executor;
+    }
+
+    public static void setTrackId(String trackId) {
+        LocalExecutor.trackId = trackId;
+    }
+
+    public static void clearTrackId() {
+        LocalExecutor.trackId = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/TupleInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/TupleInfo.java b/storm-client/src/jvm/org/apache/storm/executor/TupleInfo.java
new file mode 100644
index 0000000..4b6d0fa
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/TupleInfo.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor;
+
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.commons.lang.builder.ToStringStyle;
+
+import java.io.Serializable;
+import java.util.List;
+
+public class TupleInfo implements Serializable {
+
+    private static final long serialVersionUID = -3348670497595864118L;
+
+    private int taskId;
+    private Object messageId;
+    private String stream;
+    private List<Object> values;
+    private long timestamp;
+    private String id;
+
+    public Object getMessageId() {
+        return messageId;
+    }
+
+    public void setMessageId(Object messageId) {
+        this.messageId = messageId;
+    }
+
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    public void setTimestamp(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    public void setStream(String stream) {
+        this.stream = stream;
+    }
+
+    public List<Object> getValues() {
+        return values;
+    }
+
+    public void setValues(List<Object> values) {
+        this.values = values;
+    }
+
+    @Override
+    public String toString() {
+        return ToStringBuilder.reflectionToString(this,
+                ToStringStyle.SHORT_PREFIX_STYLE);
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public int getTaskId() {
+        return taskId;
+    }
+
+    public void setTaskId(int taskId) {
+        this.taskId = taskId;
+    }
+}


[35/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/BoltStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/BoltStats.java b/storm-client/src/jvm/org/apache/storm/generated/BoltStats.java
new file mode 100644
index 0000000..cd5a497
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/BoltStats.java
@@ -0,0 +1,1390 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._Fields>, java.io.Serializable, Cloneable, Comparable<BoltStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltStats");
+
+  private static final org.apache.thrift.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("acked", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("failed", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField PROCESS_MS_AVG_FIELD_DESC = new org.apache.thrift.protocol.TField("process_ms_avg", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField EXECUTED_FIELD_DESC = new org.apache.thrift.protocol.TField("executed", org.apache.thrift.protocol.TType.MAP, (short)4);
+  private static final org.apache.thrift.protocol.TField EXECUTE_MS_AVG_FIELD_DESC = new org.apache.thrift.protocol.TField("execute_ms_avg", org.apache.thrift.protocol.TType.MAP, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new BoltStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new BoltStatsTupleSchemeFactory());
+  }
+
+  private Map<String,Map<GlobalStreamId,Long>> acked; // required
+  private Map<String,Map<GlobalStreamId,Long>> failed; // required
+  private Map<String,Map<GlobalStreamId,Double>> process_ms_avg; // required
+  private Map<String,Map<GlobalStreamId,Long>> executed; // required
+  private Map<String,Map<GlobalStreamId,Double>> execute_ms_avg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ACKED((short)1, "acked"),
+    FAILED((short)2, "failed"),
+    PROCESS_MS_AVG((short)3, "process_ms_avg"),
+    EXECUTED((short)4, "executed"),
+    EXECUTE_MS_AVG((short)5, "execute_ms_avg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACKED
+          return ACKED;
+        case 2: // FAILED
+          return FAILED;
+        case 3: // PROCESS_MS_AVG
+          return PROCESS_MS_AVG;
+        case 4: // EXECUTED
+          return EXECUTED;
+        case 5: // EXECUTE_MS_AVG
+          return EXECUTE_MS_AVG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACKED, new org.apache.thrift.meta_data.FieldMetaData("acked", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
+    tmpMap.put(_Fields.FAILED, new org.apache.thrift.meta_data.FieldMetaData("failed", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
+    tmpMap.put(_Fields.PROCESS_MS_AVG, new org.apache.thrift.meta_data.FieldMetaData("process_ms_avg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))));
+    tmpMap.put(_Fields.EXECUTED, new org.apache.thrift.meta_data.FieldMetaData("executed", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
+    tmpMap.put(_Fields.EXECUTE_MS_AVG, new org.apache.thrift.meta_data.FieldMetaData("execute_ms_avg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(BoltStats.class, metaDataMap);
+  }
+
+  public BoltStats() {
+  }
+
+  public BoltStats(
+    Map<String,Map<GlobalStreamId,Long>> acked,
+    Map<String,Map<GlobalStreamId,Long>> failed,
+    Map<String,Map<GlobalStreamId,Double>> process_ms_avg,
+    Map<String,Map<GlobalStreamId,Long>> executed,
+    Map<String,Map<GlobalStreamId,Double>> execute_ms_avg)
+  {
+    this();
+    this.acked = acked;
+    this.failed = failed;
+    this.process_ms_avg = process_ms_avg;
+    this.executed = executed;
+    this.execute_ms_avg = execute_ms_avg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public BoltStats(BoltStats other) {
+    if (other.is_set_acked()) {
+      Map<String,Map<GlobalStreamId,Long>> __this__acked = new HashMap<String,Map<GlobalStreamId,Long>>(other.acked.size());
+      for (Map.Entry<String, Map<GlobalStreamId,Long>> other_element : other.acked.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Long> other_element_value = other_element.getValue();
+
+        String __this__acked_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Long> __this__acked_copy_value = new HashMap<GlobalStreamId,Long>(other_element_value.size());
+        for (Map.Entry<GlobalStreamId, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__acked_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Long __this__acked_copy_value_copy_value = other_element_value_element_value;
+
+          __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value);
+        }
+
+        __this__acked.put(__this__acked_copy_key, __this__acked_copy_value);
+      }
+      this.acked = __this__acked;
+    }
+    if (other.is_set_failed()) {
+      Map<String,Map<GlobalStreamId,Long>> __this__failed = new HashMap<String,Map<GlobalStreamId,Long>>(other.failed.size());
+      for (Map.Entry<String, Map<GlobalStreamId,Long>> other_element : other.failed.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Long> other_element_value = other_element.getValue();
+
+        String __this__failed_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Long> __this__failed_copy_value = new HashMap<GlobalStreamId,Long>(other_element_value.size());
+        for (Map.Entry<GlobalStreamId, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__failed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Long __this__failed_copy_value_copy_value = other_element_value_element_value;
+
+          __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value);
+        }
+
+        __this__failed.put(__this__failed_copy_key, __this__failed_copy_value);
+      }
+      this.failed = __this__failed;
+    }
+    if (other.is_set_process_ms_avg()) {
+      Map<String,Map<GlobalStreamId,Double>> __this__process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(other.process_ms_avg.size());
+      for (Map.Entry<String, Map<GlobalStreamId,Double>> other_element : other.process_ms_avg.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Double> other_element_value = other_element.getValue();
+
+        String __this__process_ms_avg_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Double> __this__process_ms_avg_copy_value = new HashMap<GlobalStreamId,Double>(other_element_value.size());
+        for (Map.Entry<GlobalStreamId, Double> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Double other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__process_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Double __this__process_ms_avg_copy_value_copy_value = other_element_value_element_value;
+
+          __this__process_ms_avg_copy_value.put(__this__process_ms_avg_copy_value_copy_key, __this__process_ms_avg_copy_value_copy_value);
+        }
+
+        __this__process_ms_avg.put(__this__process_ms_avg_copy_key, __this__process_ms_avg_copy_value);
+      }
+      this.process_ms_avg = __this__process_ms_avg;
+    }
+    if (other.is_set_executed()) {
+      Map<String,Map<GlobalStreamId,Long>> __this__executed = new HashMap<String,Map<GlobalStreamId,Long>>(other.executed.size());
+      for (Map.Entry<String, Map<GlobalStreamId,Long>> other_element : other.executed.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Long> other_element_value = other_element.getValue();
+
+        String __this__executed_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Long> __this__executed_copy_value = new HashMap<GlobalStreamId,Long>(other_element_value.size());
+        for (Map.Entry<GlobalStreamId, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__executed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Long __this__executed_copy_value_copy_value = other_element_value_element_value;
+
+          __this__executed_copy_value.put(__this__executed_copy_value_copy_key, __this__executed_copy_value_copy_value);
+        }
+
+        __this__executed.put(__this__executed_copy_key, __this__executed_copy_value);
+      }
+      this.executed = __this__executed;
+    }
+    if (other.is_set_execute_ms_avg()) {
+      Map<String,Map<GlobalStreamId,Double>> __this__execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(other.execute_ms_avg.size());
+      for (Map.Entry<String, Map<GlobalStreamId,Double>> other_element : other.execute_ms_avg.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Double> other_element_value = other_element.getValue();
+
+        String __this__execute_ms_avg_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Double> __this__execute_ms_avg_copy_value = new HashMap<GlobalStreamId,Double>(other_element_value.size());
+        for (Map.Entry<GlobalStreamId, Double> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Double other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__execute_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Double __this__execute_ms_avg_copy_value_copy_value = other_element_value_element_value;
+
+          __this__execute_ms_avg_copy_value.put(__this__execute_ms_avg_copy_value_copy_key, __this__execute_ms_avg_copy_value_copy_value);
+        }
+
+        __this__execute_ms_avg.put(__this__execute_ms_avg_copy_key, __this__execute_ms_avg_copy_value);
+      }
+      this.execute_ms_avg = __this__execute_ms_avg;
+    }
+  }
+
+  public BoltStats deepCopy() {
+    return new BoltStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.acked = null;
+    this.failed = null;
+    this.process_ms_avg = null;
+    this.executed = null;
+    this.execute_ms_avg = null;
+  }
+
+  public int get_acked_size() {
+    return (this.acked == null) ? 0 : this.acked.size();
+  }
+
+  public void put_to_acked(String key, Map<GlobalStreamId,Long> val) {
+    if (this.acked == null) {
+      this.acked = new HashMap<String,Map<GlobalStreamId,Long>>();
+    }
+    this.acked.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Long>> get_acked() {
+    return this.acked;
+  }
+
+  public void set_acked(Map<String,Map<GlobalStreamId,Long>> acked) {
+    this.acked = acked;
+  }
+
+  public void unset_acked() {
+    this.acked = null;
+  }
+
+  /** Returns true if field acked is set (has been assigned a value) and false otherwise */
+  public boolean is_set_acked() {
+    return this.acked != null;
+  }
+
+  public void set_acked_isSet(boolean value) {
+    if (!value) {
+      this.acked = null;
+    }
+  }
+
+  public int get_failed_size() {
+    return (this.failed == null) ? 0 : this.failed.size();
+  }
+
+  public void put_to_failed(String key, Map<GlobalStreamId,Long> val) {
+    if (this.failed == null) {
+      this.failed = new HashMap<String,Map<GlobalStreamId,Long>>();
+    }
+    this.failed.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Long>> get_failed() {
+    return this.failed;
+  }
+
+  public void set_failed(Map<String,Map<GlobalStreamId,Long>> failed) {
+    this.failed = failed;
+  }
+
+  public void unset_failed() {
+    this.failed = null;
+  }
+
+  /** Returns true if field failed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_failed() {
+    return this.failed != null;
+  }
+
+  public void set_failed_isSet(boolean value) {
+    if (!value) {
+      this.failed = null;
+    }
+  }
+
+  public int get_process_ms_avg_size() {
+    return (this.process_ms_avg == null) ? 0 : this.process_ms_avg.size();
+  }
+
+  public void put_to_process_ms_avg(String key, Map<GlobalStreamId,Double> val) {
+    if (this.process_ms_avg == null) {
+      this.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>();
+    }
+    this.process_ms_avg.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Double>> get_process_ms_avg() {
+    return this.process_ms_avg;
+  }
+
+  public void set_process_ms_avg(Map<String,Map<GlobalStreamId,Double>> process_ms_avg) {
+    this.process_ms_avg = process_ms_avg;
+  }
+
+  public void unset_process_ms_avg() {
+    this.process_ms_avg = null;
+  }
+
+  /** Returns true if field process_ms_avg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_process_ms_avg() {
+    return this.process_ms_avg != null;
+  }
+
+  public void set_process_ms_avg_isSet(boolean value) {
+    if (!value) {
+      this.process_ms_avg = null;
+    }
+  }
+
+  public int get_executed_size() {
+    return (this.executed == null) ? 0 : this.executed.size();
+  }
+
+  public void put_to_executed(String key, Map<GlobalStreamId,Long> val) {
+    if (this.executed == null) {
+      this.executed = new HashMap<String,Map<GlobalStreamId,Long>>();
+    }
+    this.executed.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Long>> get_executed() {
+    return this.executed;
+  }
+
+  public void set_executed(Map<String,Map<GlobalStreamId,Long>> executed) {
+    this.executed = executed;
+  }
+
+  public void unset_executed() {
+    this.executed = null;
+  }
+
+  /** Returns true if field executed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executed() {
+    return this.executed != null;
+  }
+
+  public void set_executed_isSet(boolean value) {
+    if (!value) {
+      this.executed = null;
+    }
+  }
+
+  public int get_execute_ms_avg_size() {
+    return (this.execute_ms_avg == null) ? 0 : this.execute_ms_avg.size();
+  }
+
+  public void put_to_execute_ms_avg(String key, Map<GlobalStreamId,Double> val) {
+    if (this.execute_ms_avg == null) {
+      this.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>();
+    }
+    this.execute_ms_avg.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Double>> get_execute_ms_avg() {
+    return this.execute_ms_avg;
+  }
+
+  public void set_execute_ms_avg(Map<String,Map<GlobalStreamId,Double>> execute_ms_avg) {
+    this.execute_ms_avg = execute_ms_avg;
+  }
+
+  public void unset_execute_ms_avg() {
+    this.execute_ms_avg = null;
+  }
+
+  /** Returns true if field execute_ms_avg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_execute_ms_avg() {
+    return this.execute_ms_avg != null;
+  }
+
+  public void set_execute_ms_avg_isSet(boolean value) {
+    if (!value) {
+      this.execute_ms_avg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACKED:
+      if (value == null) {
+        unset_acked();
+      } else {
+        set_acked((Map<String,Map<GlobalStreamId,Long>>)value);
+      }
+      break;
+
+    case FAILED:
+      if (value == null) {
+        unset_failed();
+      } else {
+        set_failed((Map<String,Map<GlobalStreamId,Long>>)value);
+      }
+      break;
+
+    case PROCESS_MS_AVG:
+      if (value == null) {
+        unset_process_ms_avg();
+      } else {
+        set_process_ms_avg((Map<String,Map<GlobalStreamId,Double>>)value);
+      }
+      break;
+
+    case EXECUTED:
+      if (value == null) {
+        unset_executed();
+      } else {
+        set_executed((Map<String,Map<GlobalStreamId,Long>>)value);
+      }
+      break;
+
+    case EXECUTE_MS_AVG:
+      if (value == null) {
+        unset_execute_ms_avg();
+      } else {
+        set_execute_ms_avg((Map<String,Map<GlobalStreamId,Double>>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACKED:
+      return get_acked();
+
+    case FAILED:
+      return get_failed();
+
+    case PROCESS_MS_AVG:
+      return get_process_ms_avg();
+
+    case EXECUTED:
+      return get_executed();
+
+    case EXECUTE_MS_AVG:
+      return get_execute_ms_avg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ACKED:
+      return is_set_acked();
+    case FAILED:
+      return is_set_failed();
+    case PROCESS_MS_AVG:
+      return is_set_process_ms_avg();
+    case EXECUTED:
+      return is_set_executed();
+    case EXECUTE_MS_AVG:
+      return is_set_execute_ms_avg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof BoltStats)
+      return this.equals((BoltStats)that);
+    return false;
+  }
+
+  public boolean equals(BoltStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_acked = true && this.is_set_acked();
+    boolean that_present_acked = true && that.is_set_acked();
+    if (this_present_acked || that_present_acked) {
+      if (!(this_present_acked && that_present_acked))
+        return false;
+      if (!this.acked.equals(that.acked))
+        return false;
+    }
+
+    boolean this_present_failed = true && this.is_set_failed();
+    boolean that_present_failed = true && that.is_set_failed();
+    if (this_present_failed || that_present_failed) {
+      if (!(this_present_failed && that_present_failed))
+        return false;
+      if (!this.failed.equals(that.failed))
+        return false;
+    }
+
+    boolean this_present_process_ms_avg = true && this.is_set_process_ms_avg();
+    boolean that_present_process_ms_avg = true && that.is_set_process_ms_avg();
+    if (this_present_process_ms_avg || that_present_process_ms_avg) {
+      if (!(this_present_process_ms_avg && that_present_process_ms_avg))
+        return false;
+      if (!this.process_ms_avg.equals(that.process_ms_avg))
+        return false;
+    }
+
+    boolean this_present_executed = true && this.is_set_executed();
+    boolean that_present_executed = true && that.is_set_executed();
+    if (this_present_executed || that_present_executed) {
+      if (!(this_present_executed && that_present_executed))
+        return false;
+      if (!this.executed.equals(that.executed))
+        return false;
+    }
+
+    boolean this_present_execute_ms_avg = true && this.is_set_execute_ms_avg();
+    boolean that_present_execute_ms_avg = true && that.is_set_execute_ms_avg();
+    if (this_present_execute_ms_avg || that_present_execute_ms_avg) {
+      if (!(this_present_execute_ms_avg && that_present_execute_ms_avg))
+        return false;
+      if (!this.execute_ms_avg.equals(that.execute_ms_avg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_acked = true && (is_set_acked());
+    list.add(present_acked);
+    if (present_acked)
+      list.add(acked);
+
+    boolean present_failed = true && (is_set_failed());
+    list.add(present_failed);
+    if (present_failed)
+      list.add(failed);
+
+    boolean present_process_ms_avg = true && (is_set_process_ms_avg());
+    list.add(present_process_ms_avg);
+    if (present_process_ms_avg)
+      list.add(process_ms_avg);
+
+    boolean present_executed = true && (is_set_executed());
+    list.add(present_executed);
+    if (present_executed)
+      list.add(executed);
+
+    boolean present_execute_ms_avg = true && (is_set_execute_ms_avg());
+    list.add(present_execute_ms_avg);
+    if (present_execute_ms_avg)
+      list.add(execute_ms_avg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(BoltStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_acked()).compareTo(other.is_set_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acked()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acked, other.acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_failed()).compareTo(other.is_set_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_failed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.failed, other.failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_process_ms_avg()).compareTo(other.is_set_process_ms_avg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_process_ms_avg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.process_ms_avg, other.process_ms_avg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executed()).compareTo(other.is_set_executed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executed, other.executed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_execute_ms_avg()).compareTo(other.is_set_execute_ms_avg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_execute_ms_avg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.execute_ms_avg, other.execute_ms_avg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("BoltStats(");
+    boolean first = true;
+
+    sb.append("acked:");
+    if (this.acked == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.acked);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("failed:");
+    if (this.failed == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.failed);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("process_ms_avg:");
+    if (this.process_ms_avg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.process_ms_avg);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executed:");
+    if (this.executed == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executed);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("execute_ms_avg:");
+    if (this.execute_ms_avg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.execute_ms_avg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_acked()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'acked' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_failed()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'failed' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_process_ms_avg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'process_ms_avg' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executed()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executed' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_execute_ms_avg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'execute_ms_avg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class BoltStatsStandardSchemeFactory implements SchemeFactory {
+    public BoltStatsStandardScheme getScheme() {
+      return new BoltStatsStandardScheme();
+    }
+  }
+
+  private static class BoltStatsStandardScheme extends StandardScheme<BoltStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, BoltStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ACKED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map132 = iprot.readMapBegin();
+                struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map132.size);
+                String _key133;
+                Map<GlobalStreamId,Long> _val134;
+                for (int _i135 = 0; _i135 < _map132.size; ++_i135)
+                {
+                  _key133 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map136 = iprot.readMapBegin();
+                    _val134 = new HashMap<GlobalStreamId,Long>(2*_map136.size);
+                    GlobalStreamId _key137;
+                    long _val138;
+                    for (int _i139 = 0; _i139 < _map136.size; ++_i139)
+                    {
+                      _key137 = new GlobalStreamId();
+                      _key137.read(iprot);
+                      _val138 = iprot.readI64();
+                      _val134.put(_key137, _val138);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.acked.put(_key133, _val134);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_acked_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // FAILED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map140 = iprot.readMapBegin();
+                struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map140.size);
+                String _key141;
+                Map<GlobalStreamId,Long> _val142;
+                for (int _i143 = 0; _i143 < _map140.size; ++_i143)
+                {
+                  _key141 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map144 = iprot.readMapBegin();
+                    _val142 = new HashMap<GlobalStreamId,Long>(2*_map144.size);
+                    GlobalStreamId _key145;
+                    long _val146;
+                    for (int _i147 = 0; _i147 < _map144.size; ++_i147)
+                    {
+                      _key145 = new GlobalStreamId();
+                      _key145.read(iprot);
+                      _val146 = iprot.readI64();
+                      _val142.put(_key145, _val146);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.failed.put(_key141, _val142);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_failed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // PROCESS_MS_AVG
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map148 = iprot.readMapBegin();
+                struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map148.size);
+                String _key149;
+                Map<GlobalStreamId,Double> _val150;
+                for (int _i151 = 0; _i151 < _map148.size; ++_i151)
+                {
+                  _key149 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map152 = iprot.readMapBegin();
+                    _val150 = new HashMap<GlobalStreamId,Double>(2*_map152.size);
+                    GlobalStreamId _key153;
+                    double _val154;
+                    for (int _i155 = 0; _i155 < _map152.size; ++_i155)
+                    {
+                      _key153 = new GlobalStreamId();
+                      _key153.read(iprot);
+                      _val154 = iprot.readDouble();
+                      _val150.put(_key153, _val154);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.process_ms_avg.put(_key149, _val150);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_process_ms_avg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // EXECUTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map156 = iprot.readMapBegin();
+                struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map156.size);
+                String _key157;
+                Map<GlobalStreamId,Long> _val158;
+                for (int _i159 = 0; _i159 < _map156.size; ++_i159)
+                {
+                  _key157 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map160 = iprot.readMapBegin();
+                    _val158 = new HashMap<GlobalStreamId,Long>(2*_map160.size);
+                    GlobalStreamId _key161;
+                    long _val162;
+                    for (int _i163 = 0; _i163 < _map160.size; ++_i163)
+                    {
+                      _key161 = new GlobalStreamId();
+                      _key161.read(iprot);
+                      _val162 = iprot.readI64();
+                      _val158.put(_key161, _val162);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.executed.put(_key157, _val158);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // EXECUTE_MS_AVG
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map164 = iprot.readMapBegin();
+                struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map164.size);
+                String _key165;
+                Map<GlobalStreamId,Double> _val166;
+                for (int _i167 = 0; _i167 < _map164.size; ++_i167)
+                {
+                  _key165 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map168 = iprot.readMapBegin();
+                    _val166 = new HashMap<GlobalStreamId,Double>(2*_map168.size);
+                    GlobalStreamId _key169;
+                    double _val170;
+                    for (int _i171 = 0; _i171 < _map168.size; ++_i171)
+                    {
+                      _key169 = new GlobalStreamId();
+                      _key169.read(iprot);
+                      _val170 = iprot.readDouble();
+                      _val166.put(_key169, _val170);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.execute_ms_avg.put(_key165, _val166);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_execute_ms_avg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, BoltStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.acked != null) {
+        oprot.writeFieldBegin(ACKED_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter172 : struct.acked.entrySet())
+          {
+            oprot.writeString(_iter172.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter172.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter173 : _iter172.getValue().entrySet())
+              {
+                _iter173.getKey().write(oprot);
+                oprot.writeI64(_iter173.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.failed != null) {
+        oprot.writeFieldBegin(FAILED_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter174 : struct.failed.entrySet())
+          {
+            oprot.writeString(_iter174.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter174.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter175 : _iter174.getValue().entrySet())
+              {
+                _iter175.getKey().write(oprot);
+                oprot.writeI64(_iter175.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.process_ms_avg != null) {
+        oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.process_ms_avg.size()));
+          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter176 : struct.process_ms_avg.entrySet())
+          {
+            oprot.writeString(_iter176.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter176.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Double> _iter177 : _iter176.getValue().entrySet())
+              {
+                _iter177.getKey().write(oprot);
+                oprot.writeDouble(_iter177.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.executed != null) {
+        oprot.writeFieldBegin(EXECUTED_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.executed.size()));
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter178 : struct.executed.entrySet())
+          {
+            oprot.writeString(_iter178.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter178.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter179 : _iter178.getValue().entrySet())
+              {
+                _iter179.getKey().write(oprot);
+                oprot.writeI64(_iter179.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.execute_ms_avg != null) {
+        oprot.writeFieldBegin(EXECUTE_MS_AVG_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.execute_ms_avg.size()));
+          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter180 : struct.execute_ms_avg.entrySet())
+          {
+            oprot.writeString(_iter180.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter180.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Double> _iter181 : _iter180.getValue().entrySet())
+              {
+                _iter181.getKey().write(oprot);
+                oprot.writeDouble(_iter181.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class BoltStatsTupleSchemeFactory implements SchemeFactory {
+    public BoltStatsTupleScheme getScheme() {
+      return new BoltStatsTupleScheme();
+    }
+  }
+
+  private static class BoltStatsTupleScheme extends TupleScheme<BoltStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, BoltStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.acked.size());
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter182 : struct.acked.entrySet())
+        {
+          oprot.writeString(_iter182.getKey());
+          {
+            oprot.writeI32(_iter182.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter183 : _iter182.getValue().entrySet())
+            {
+              _iter183.getKey().write(oprot);
+              oprot.writeI64(_iter183.getValue());
+            }
+          }
+        }
+      }
+      {
+        oprot.writeI32(struct.failed.size());
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter184 : struct.failed.entrySet())
+        {
+          oprot.writeString(_iter184.getKey());
+          {
+            oprot.writeI32(_iter184.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter185 : _iter184.getValue().entrySet())
+            {
+              _iter185.getKey().write(oprot);
+              oprot.writeI64(_iter185.getValue());
+            }
+          }
+        }
+      }
+      {
+        oprot.writeI32(struct.process_ms_avg.size());
+        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter186 : struct.process_ms_avg.entrySet())
+        {
+          oprot.writeString(_iter186.getKey());
+          {
+            oprot.writeI32(_iter186.getValue().size());
+            for (Map.Entry<GlobalStreamId, Double> _iter187 : _iter186.getValue().entrySet())
+            {
+              _iter187.getKey().write(oprot);
+              oprot.writeDouble(_iter187.getValue());
+            }
+          }
+        }
+      }
+      {
+        oprot.writeI32(struct.executed.size());
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter188 : struct.executed.entrySet())
+        {
+          oprot.writeString(_iter188.getKey());
+          {
+            oprot.writeI32(_iter188.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter189 : _iter188.getValue().entrySet())
+            {
+              _iter189.getKey().write(oprot);
+              oprot.writeI64(_iter189.getValue());
+            }
+          }
+        }
+      }
+      {
+        oprot.writeI32(struct.execute_ms_avg.size());
+        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter190 : struct.execute_ms_avg.entrySet())
+        {
+          oprot.writeString(_iter190.getKey());
+          {
+            oprot.writeI32(_iter190.getValue().size());
+            for (Map.Entry<GlobalStreamId, Double> _iter191 : _iter190.getValue().entrySet())
+            {
+              _iter191.getKey().write(oprot);
+              oprot.writeDouble(_iter191.getValue());
+            }
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, BoltStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map192 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map192.size);
+        String _key193;
+        Map<GlobalStreamId,Long> _val194;
+        for (int _i195 = 0; _i195 < _map192.size; ++_i195)
+        {
+          _key193 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map196 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val194 = new HashMap<GlobalStreamId,Long>(2*_map196.size);
+            GlobalStreamId _key197;
+            long _val198;
+            for (int _i199 = 0; _i199 < _map196.size; ++_i199)
+            {
+              _key197 = new GlobalStreamId();
+              _key197.read(iprot);
+              _val198 = iprot.readI64();
+              _val194.put(_key197, _val198);
+            }
+          }
+          struct.acked.put(_key193, _val194);
+        }
+      }
+      struct.set_acked_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map200 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map200.size);
+        String _key201;
+        Map<GlobalStreamId,Long> _val202;
+        for (int _i203 = 0; _i203 < _map200.size; ++_i203)
+        {
+          _key201 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map204 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val202 = new HashMap<GlobalStreamId,Long>(2*_map204.size);
+            GlobalStreamId _key205;
+            long _val206;
+            for (int _i207 = 0; _i207 < _map204.size; ++_i207)
+            {
+              _key205 = new GlobalStreamId();
+              _key205.read(iprot);
+              _val206 = iprot.readI64();
+              _val202.put(_key205, _val206);
+            }
+          }
+          struct.failed.put(_key201, _val202);
+        }
+      }
+      struct.set_failed_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map208 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map208.size);
+        String _key209;
+        Map<GlobalStreamId,Double> _val210;
+        for (int _i211 = 0; _i211 < _map208.size; ++_i211)
+        {
+          _key209 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map212 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val210 = new HashMap<GlobalStreamId,Double>(2*_map212.size);
+            GlobalStreamId _key213;
+            double _val214;
+            for (int _i215 = 0; _i215 < _map212.size; ++_i215)
+            {
+              _key213 = new GlobalStreamId();
+              _key213.read(iprot);
+              _val214 = iprot.readDouble();
+              _val210.put(_key213, _val214);
+            }
+          }
+          struct.process_ms_avg.put(_key209, _val210);
+        }
+      }
+      struct.set_process_ms_avg_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map216 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map216.size);
+        String _key217;
+        Map<GlobalStreamId,Long> _val218;
+        for (int _i219 = 0; _i219 < _map216.size; ++_i219)
+        {
+          _key217 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map220 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val218 = new HashMap<GlobalStreamId,Long>(2*_map220.size);
+            GlobalStreamId _key221;
+            long _val222;
+            for (int _i223 = 0; _i223 < _map220.size; ++_i223)
+            {
+              _key221 = new GlobalStreamId();
+              _key221.read(iprot);
+              _val222 = iprot.readI64();
+              _val218.put(_key221, _val222);
+            }
+          }
+          struct.executed.put(_key217, _val218);
+        }
+      }
+      struct.set_executed_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map224 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map224.size);
+        String _key225;
+        Map<GlobalStreamId,Double> _val226;
+        for (int _i227 = 0; _i227 < _map224.size; ++_i227)
+        {
+          _key225 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map228 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val226 = new HashMap<GlobalStreamId,Double>(2*_map228.size);
+            GlobalStreamId _key229;
+            double _val230;
+            for (int _i231 = 0; _i231 < _map228.size; ++_i231)
+            {
+              _key229 = new GlobalStreamId();
+              _key229.read(iprot);
+              _val230 = iprot.readDouble();
+              _val226.put(_key229, _val230);
+            }
+          }
+          struct.execute_ms_avg.put(_key225, _val226);
+        }
+      }
+      struct.set_execute_ms_avg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ClusterSummary.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ClusterSummary.java b/storm-client/src/jvm/org/apache/storm/generated/ClusterSummary.java
new file mode 100644
index 0000000..8adc3f5
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ClusterSummary.java
@@ -0,0 +1,879 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, ClusterSummary._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterSummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterSummary");
+
+  private static final org.apache.thrift.protocol.TField SUPERVISORS_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisors", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField NIMBUS_UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus_uptime_secs", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField TOPOLOGIES_FIELD_DESC = new org.apache.thrift.protocol.TField("topologies", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField NIMBUSES_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbuses", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ClusterSummaryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ClusterSummaryTupleSchemeFactory());
+  }
+
+  private List<SupervisorSummary> supervisors; // required
+  private int nimbus_uptime_secs; // optional
+  private List<TopologySummary> topologies; // required
+  private List<NimbusSummary> nimbuses; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SUPERVISORS((short)1, "supervisors"),
+    NIMBUS_UPTIME_SECS((short)2, "nimbus_uptime_secs"),
+    TOPOLOGIES((short)3, "topologies"),
+    NIMBUSES((short)4, "nimbuses");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SUPERVISORS
+          return SUPERVISORS;
+        case 2: // NIMBUS_UPTIME_SECS
+          return NIMBUS_UPTIME_SECS;
+        case 3: // TOPOLOGIES
+          return TOPOLOGIES;
+        case 4: // NIMBUSES
+          return NIMBUSES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NIMBUS_UPTIME_SECS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NIMBUS_UPTIME_SECS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SUPERVISORS, new org.apache.thrift.meta_data.FieldMetaData("supervisors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorSummary.class))));
+    tmpMap.put(_Fields.NIMBUS_UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("nimbus_uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.TOPOLOGIES, new org.apache.thrift.meta_data.FieldMetaData("topologies", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologySummary.class))));
+    tmpMap.put(_Fields.NIMBUSES, new org.apache.thrift.meta_data.FieldMetaData("nimbuses", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NimbusSummary.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClusterSummary.class, metaDataMap);
+  }
+
+  public ClusterSummary() {
+    this.nimbus_uptime_secs = 0;
+
+  }
+
+  public ClusterSummary(
+    List<SupervisorSummary> supervisors,
+    List<TopologySummary> topologies,
+    List<NimbusSummary> nimbuses)
+  {
+    this();
+    this.supervisors = supervisors;
+    this.topologies = topologies;
+    this.nimbuses = nimbuses;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ClusterSummary(ClusterSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_supervisors()) {
+      List<SupervisorSummary> __this__supervisors = new ArrayList<SupervisorSummary>(other.supervisors.size());
+      for (SupervisorSummary other_element : other.supervisors) {
+        __this__supervisors.add(new SupervisorSummary(other_element));
+      }
+      this.supervisors = __this__supervisors;
+    }
+    this.nimbus_uptime_secs = other.nimbus_uptime_secs;
+    if (other.is_set_topologies()) {
+      List<TopologySummary> __this__topologies = new ArrayList<TopologySummary>(other.topologies.size());
+      for (TopologySummary other_element : other.topologies) {
+        __this__topologies.add(new TopologySummary(other_element));
+      }
+      this.topologies = __this__topologies;
+    }
+    if (other.is_set_nimbuses()) {
+      List<NimbusSummary> __this__nimbuses = new ArrayList<NimbusSummary>(other.nimbuses.size());
+      for (NimbusSummary other_element : other.nimbuses) {
+        __this__nimbuses.add(new NimbusSummary(other_element));
+      }
+      this.nimbuses = __this__nimbuses;
+    }
+  }
+
+  public ClusterSummary deepCopy() {
+    return new ClusterSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.supervisors = null;
+    this.nimbus_uptime_secs = 0;
+
+    this.topologies = null;
+    this.nimbuses = null;
+  }
+
+  public int get_supervisors_size() {
+    return (this.supervisors == null) ? 0 : this.supervisors.size();
+  }
+
+  public java.util.Iterator<SupervisorSummary> get_supervisors_iterator() {
+    return (this.supervisors == null) ? null : this.supervisors.iterator();
+  }
+
+  public void add_to_supervisors(SupervisorSummary elem) {
+    if (this.supervisors == null) {
+      this.supervisors = new ArrayList<SupervisorSummary>();
+    }
+    this.supervisors.add(elem);
+  }
+
+  public List<SupervisorSummary> get_supervisors() {
+    return this.supervisors;
+  }
+
+  public void set_supervisors(List<SupervisorSummary> supervisors) {
+    this.supervisors = supervisors;
+  }
+
+  public void unset_supervisors() {
+    this.supervisors = null;
+  }
+
+  /** Returns true if field supervisors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_supervisors() {
+    return this.supervisors != null;
+  }
+
+  public void set_supervisors_isSet(boolean value) {
+    if (!value) {
+      this.supervisors = null;
+    }
+  }
+
+  public int get_nimbus_uptime_secs() {
+    return this.nimbus_uptime_secs;
+  }
+
+  public void set_nimbus_uptime_secs(int nimbus_uptime_secs) {
+    this.nimbus_uptime_secs = nimbus_uptime_secs;
+    set_nimbus_uptime_secs_isSet(true);
+  }
+
+  public void unset_nimbus_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field nimbus_uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_nimbus_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_nimbus_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_topologies_size() {
+    return (this.topologies == null) ? 0 : this.topologies.size();
+  }
+
+  public java.util.Iterator<TopologySummary> get_topologies_iterator() {
+    return (this.topologies == null) ? null : this.topologies.iterator();
+  }
+
+  public void add_to_topologies(TopologySummary elem) {
+    if (this.topologies == null) {
+      this.topologies = new ArrayList<TopologySummary>();
+    }
+    this.topologies.add(elem);
+  }
+
+  public List<TopologySummary> get_topologies() {
+    return this.topologies;
+  }
+
+  public void set_topologies(List<TopologySummary> topologies) {
+    this.topologies = topologies;
+  }
+
+  public void unset_topologies() {
+    this.topologies = null;
+  }
+
+  /** Returns true if field topologies is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topologies() {
+    return this.topologies != null;
+  }
+
+  public void set_topologies_isSet(boolean value) {
+    if (!value) {
+      this.topologies = null;
+    }
+  }
+
+  public int get_nimbuses_size() {
+    return (this.nimbuses == null) ? 0 : this.nimbuses.size();
+  }
+
+  public java.util.Iterator<NimbusSummary> get_nimbuses_iterator() {
+    return (this.nimbuses == null) ? null : this.nimbuses.iterator();
+  }
+
+  public void add_to_nimbuses(NimbusSummary elem) {
+    if (this.nimbuses == null) {
+      this.nimbuses = new ArrayList<NimbusSummary>();
+    }
+    this.nimbuses.add(elem);
+  }
+
+  public List<NimbusSummary> get_nimbuses() {
+    return this.nimbuses;
+  }
+
+  public void set_nimbuses(List<NimbusSummary> nimbuses) {
+    this.nimbuses = nimbuses;
+  }
+
+  public void unset_nimbuses() {
+    this.nimbuses = null;
+  }
+
+  /** Returns true if field nimbuses is set (has been assigned a value) and false otherwise */
+  public boolean is_set_nimbuses() {
+    return this.nimbuses != null;
+  }
+
+  public void set_nimbuses_isSet(boolean value) {
+    if (!value) {
+      this.nimbuses = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SUPERVISORS:
+      if (value == null) {
+        unset_supervisors();
+      } else {
+        set_supervisors((List<SupervisorSummary>)value);
+      }
+      break;
+
+    case NIMBUS_UPTIME_SECS:
+      if (value == null) {
+        unset_nimbus_uptime_secs();
+      } else {
+        set_nimbus_uptime_secs((Integer)value);
+      }
+      break;
+
+    case TOPOLOGIES:
+      if (value == null) {
+        unset_topologies();
+      } else {
+        set_topologies((List<TopologySummary>)value);
+      }
+      break;
+
+    case NIMBUSES:
+      if (value == null) {
+        unset_nimbuses();
+      } else {
+        set_nimbuses((List<NimbusSummary>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SUPERVISORS:
+      return get_supervisors();
+
+    case NIMBUS_UPTIME_SECS:
+      return get_nimbus_uptime_secs();
+
+    case TOPOLOGIES:
+      return get_topologies();
+
+    case NIMBUSES:
+      return get_nimbuses();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SUPERVISORS:
+      return is_set_supervisors();
+    case NIMBUS_UPTIME_SECS:
+      return is_set_nimbus_uptime_secs();
+    case TOPOLOGIES:
+      return is_set_topologies();
+    case NIMBUSES:
+      return is_set_nimbuses();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ClusterSummary)
+      return this.equals((ClusterSummary)that);
+    return false;
+  }
+
+  public boolean equals(ClusterSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_supervisors = true && this.is_set_supervisors();
+    boolean that_present_supervisors = true && that.is_set_supervisors();
+    if (this_present_supervisors || that_present_supervisors) {
+      if (!(this_present_supervisors && that_present_supervisors))
+        return false;
+      if (!this.supervisors.equals(that.supervisors))
+        return false;
+    }
+
+    boolean this_present_nimbus_uptime_secs = true && this.is_set_nimbus_uptime_secs();
+    boolean that_present_nimbus_uptime_secs = true && that.is_set_nimbus_uptime_secs();
+    if (this_present_nimbus_uptime_secs || that_present_nimbus_uptime_secs) {
+      if (!(this_present_nimbus_uptime_secs && that_present_nimbus_uptime_secs))
+        return false;
+      if (this.nimbus_uptime_secs != that.nimbus_uptime_secs)
+        return false;
+    }
+
+    boolean this_present_topologies = true && this.is_set_topologies();
+    boolean that_present_topologies = true && that.is_set_topologies();
+    if (this_present_topologies || that_present_topologies) {
+      if (!(this_present_topologies && that_present_topologies))
+        return false;
+      if (!this.topologies.equals(that.topologies))
+        return false;
+    }
+
+    boolean this_present_nimbuses = true && this.is_set_nimbuses();
+    boolean that_present_nimbuses = true && that.is_set_nimbuses();
+    if (this_present_nimbuses || that_present_nimbuses) {
+      if (!(this_present_nimbuses && that_present_nimbuses))
+        return false;
+      if (!this.nimbuses.equals(that.nimbuses))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_supervisors = true && (is_set_supervisors());
+    list.add(present_supervisors);
+    if (present_supervisors)
+      list.add(supervisors);
+
+    boolean present_nimbus_uptime_secs = true && (is_set_nimbus_uptime_secs());
+    list.add(present_nimbus_uptime_secs);
+    if (present_nimbus_uptime_secs)
+      list.add(nimbus_uptime_secs);
+
+    boolean present_topologies = true && (is_set_topologies());
+    list.add(present_topologies);
+    if (present_topologies)
+      list.add(topologies);
+
+    boolean present_nimbuses = true && (is_set_nimbuses());
+    list.add(present_nimbuses);
+    if (present_nimbuses)
+      list.add(nimbuses);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ClusterSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_supervisors()).compareTo(other.is_set_supervisors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_supervisors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisors, other.supervisors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_nimbus_uptime_secs()).compareTo(other.is_set_nimbus_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_nimbus_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus_uptime_secs, other.nimbus_uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(other.is_set_topologies());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topologies()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, other.topologies);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_nimbuses()).compareTo(other.is_set_nimbuses());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_nimbuses()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbuses, other.nimbuses);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ClusterSummary(");
+    boolean first = true;
+
+    sb.append("supervisors:");
+    if (this.supervisors == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.supervisors);
+    }
+    first = false;
+    if (is_set_nimbus_uptime_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("nimbus_uptime_secs:");
+      sb.append(this.nimbus_uptime_secs);
+      first = false;
+    }
+    if (!first) sb.append(", ");
+    sb.append("topologies:");
+    if (this.topologies == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topologies);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("nimbuses:");
+    if (this.nimbuses == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.nimbuses);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_supervisors()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisors' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_topologies()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topologies' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_nimbuses()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbuses' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ClusterSummaryStandardSchemeFactory implements SchemeFactory {
+    public ClusterSummaryStandardScheme getScheme() {
+      return new ClusterSummaryStandardScheme();
+    }
+  }
+
+  private static class ClusterSummaryStandardScheme extends StandardScheme<ClusterSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ClusterSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SUPERVISORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list108 = iprot.readListBegin();
+                struct.supervisors = new ArrayList<SupervisorSummary>(_list108.size);
+                SupervisorSummary _elem109;
+                for (int _i110 = 0; _i110 < _list108.size; ++_i110)
+                {
+                  _elem109 = new SupervisorSummary();
+                  _elem109.read(iprot);
+                  struct.supervisors.add(_elem109);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_supervisors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NIMBUS_UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.nimbus_uptime_secs = iprot.readI32();
+              struct.set_nimbus_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TOPOLOGIES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list111 = iprot.readListBegin();
+                struct.topologies = new ArrayList<TopologySummary>(_list111.size);
+                TopologySummary _elem112;
+                for (int _i113 = 0; _i113 < _list111.size; ++_i113)
+                {
+                  _elem112 = new TopologySummary();
+                  _elem112.read(iprot);
+                  struct.topologies.add(_elem112);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_topologies_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // NIMBUSES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list114 = iprot.readListBegin();
+                struct.nimbuses = new ArrayList<NimbusSummary>(_list114.size);
+                NimbusSummary _elem115;
+                for (int _i116 = 0; _i116 < _list114.size; ++_i116)
+                {
+                  _elem115 = new NimbusSummary();
+                  _elem115.read(iprot);
+                  struct.nimbuses.add(_elem115);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_nimbuses_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ClusterSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.supervisors != null) {
+        oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.supervisors.size()));
+          for (SupervisorSummary _iter117 : struct.supervisors)
+          {
+            _iter117.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_nimbus_uptime_secs()) {
+        oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.nimbus_uptime_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.topologies != null) {
+        oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topologies.size()));
+          for (TopologySummary _iter118 : struct.topologies)
+          {
+            _iter118.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.nimbuses != null) {
+        oprot.writeFieldBegin(NIMBUSES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.nimbuses.size()));
+          for (NimbusSummary _iter119 : struct.nimbuses)
+          {
+            _iter119.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ClusterSummaryTupleSchemeFactory implements SchemeFactory {
+    public ClusterSummaryTupleScheme getScheme() {
+      return new ClusterSummaryTupleScheme();
+    }
+  }
+
+  private static class ClusterSummaryTupleScheme extends TupleScheme<ClusterSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.supervisors.size());
+        for (SupervisorSummary _iter120 : struct.supervisors)
+        {
+          _iter120.write(oprot);
+        }
+      }
+      {
+        oprot.writeI32(struct.topologies.size());
+        for (TopologySummary _iter121 : struct.topologies)
+        {
+          _iter121.write(oprot);
+        }
+      }
+      {
+        oprot.writeI32(struct.nimbuses.size());
+        for (NimbusSummary _iter122 : struct.nimbuses)
+        {
+          _iter122.write(oprot);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_nimbus_uptime_secs()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_nimbus_uptime_secs()) {
+        oprot.writeI32(struct.nimbus_uptime_secs);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list123 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.supervisors = new ArrayList<SupervisorSummary>(_list123.size);
+        SupervisorSummary _elem124;
+        for (int _i125 = 0; _i125 < _list123.size; ++_i125)
+        {
+          _elem124 = new SupervisorSummary();
+          _elem124.read(iprot);
+          struct.supervisors.add(_elem124);
+        }
+      }
+      struct.set_supervisors_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list126 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topologies = new ArrayList<TopologySummary>(_list126.size);
+        TopologySummary _elem127;
+        for (int _i128 = 0; _i128 < _list126.size; ++_i128)
+        {
+          _elem127 = new TopologySummary();
+          _elem127.read(iprot);
+          struct.topologies.add(_elem127);
+        }
+      }
+      struct.set_topologies_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list129 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.nimbuses = new ArrayList<NimbusSummary>(_list129.size);
+        NimbusSummary _elem130;
+        for (int _i131 = 0; _i131 < _list129.size; ++_i131)
+        {
+          _elem130 = new NimbusSummary();
+          _elem130.read(iprot);
+          struct.nimbuses.add(_elem130);
+        }
+      }
+      struct.set_nimbuses_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.nimbus_uptime_secs = iprot.readI32();
+        struct.set_nimbus_uptime_secs_isSet(true);
+      }
+    }
+  }
+
+}
+


[48/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
index 6e03bcf..f66c9b2 100644
--- a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
+++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
@@ -23,15 +23,12 @@ import org.apache.storm.blobstore.BlobStore;
 import org.apache.storm.blobstore.BlobStoreAclHandler;
 import org.apache.storm.generated.AccessControl;
 import org.apache.storm.generated.AuthorizationException;
-import org.apache.storm.generated.KeyAlreadyExistsException;
 import org.apache.storm.generated.KeyNotFoundException;
-import org.apache.storm.generated.ReadableBlobMeta;
 import org.apache.storm.generated.SettableBlobMeta;
 import org.apache.storm.generated.AccessControlType;
 
 import org.apache.storm.security.auth.NimbusPrincipal;
 import org.apache.storm.security.auth.SingleUserPrincipal;
-import org.apache.storm.utils.Utils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -39,7 +36,6 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -56,12 +52,10 @@ import java.util.Set;
 import java.util.Iterator;
 import java.util.Arrays;
 import java.util.List;
-import java.util.ArrayList;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Mockito.*;
 
 public class BlobStoreTest {
   private static final Logger LOG = LoggerFactory.getLogger(BlobStoreTest.class);

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-hive/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hive/pom.xml b/external/storm-hive/pom.xml
index d4d076d..ebd80c6 100644
--- a/external/storm-hive/pom.xml
+++ b/external/storm-hive/pom.xml
@@ -40,13 +40,13 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.storm</groupId>
-      <artifactId>storm-core</artifactId>
+      <artifactId>storm-client</artifactId>
       <version>${project.version}</version>
       <scope>${provided.scope}</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.storm</groupId>
-      <artifactId>storm-core</artifactId>
+      <artifactId>storm-client</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-jdbc/pom.xml b/external/storm-jdbc/pom.xml
index 1ab2e1c..46547e1 100644
--- a/external/storm-jdbc/pom.xml
+++ b/external/storm-jdbc/pom.xml
@@ -42,7 +42,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-jms/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-jms/pom.xml b/external/storm-jms/pom.xml
index 0e0feb4..3cc5bfc 100644
--- a/external/storm-jms/pom.xml
+++ b/external/storm-jms/pom.xml
@@ -42,7 +42,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <!-- keep storm out of the jar-with-dependencies -->
             <scope>${provided.scope}</scope>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-jms/src/main/java/org/apache/storm/jms/spout/JmsSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-jms/src/main/java/org/apache/storm/jms/spout/JmsSpout.java b/external/storm-jms/src/main/java/org/apache/storm/jms/spout/JmsSpout.java
index 6aaa7c9..e69ff3a 100644
--- a/external/storm-jms/src/main/java/org/apache/storm/jms/spout/JmsSpout.java
+++ b/external/storm-jms/src/main/java/org/apache/storm/jms/spout/JmsSpout.java
@@ -31,6 +31,7 @@ import javax.jms.MessageListener;
 import javax.jms.Session;
 
 import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,7 +41,6 @@ import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
 
 /**
  * A Storm <code>Spout</code> implementation that listens to a JMS topic or queue

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-jms/src/main/java/org/apache/storm/jms/trident/TridentJmsSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-jms/src/main/java/org/apache/storm/jms/trident/TridentJmsSpout.java b/external/storm-jms/src/main/java/org/apache/storm/jms/trident/TridentJmsSpout.java
index 55e29bc..96e00ad 100644
--- a/external/storm-jms/src/main/java/org/apache/storm/jms/trident/TridentJmsSpout.java
+++ b/external/storm-jms/src/main/java/org/apache/storm/jms/trident/TridentJmsSpout.java
@@ -33,6 +33,7 @@ import javax.jms.Session;
 
 import org.apache.storm.jms.JmsProvider;
 import org.apache.storm.jms.JmsTupleProducer;
+import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,7 +47,6 @@ import org.apache.storm.topology.OutputFieldsGetter;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 import org.apache.storm.utils.RotatingMap;
-import org.apache.storm.utils.Utils;
 
 /**
  * Trident implementation of the JmsSpout

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kafka-client/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/pom.xml b/external/storm-kafka-client/pom.xml
index 02c2823..a693c7d 100644
--- a/external/storm-kafka-client/pom.xml
+++ b/external/storm-kafka-client/pom.xml
@@ -44,10 +44,17 @@
         <!--parent module dependency-->
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-server</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
         <!--kafka libraries-->
         <dependency>
             <groupId>org.apache.kafka</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml
index 6571a21..a595e14 100644
--- a/external/storm-kafka/pom.xml
+++ b/external/storm-kafka/pom.xml
@@ -124,7 +124,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java
index 425d5c1..13e676a 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java
@@ -18,7 +18,7 @@
 package org.apache.storm.kafka;
 
 import org.apache.storm.Config;
-import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -55,14 +55,14 @@ public class DynamicBrokersReader {
 
         _zkPath = zkPath;
         _topic = topic;
-        _isWildcardTopic = Utils.getBoolean(conf.get("kafka.topic.wildcard.match"), false);
+        _isWildcardTopic = ObjectReader.getBoolean(conf.get("kafka.topic.wildcard.match"), false);
         try {
             _curator = CuratorFrameworkFactory.newClient(
                     zkStr,
-                    Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)),
-                    Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)),
-                    new RetryNTimes(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)),
-                            Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL))));
+                    ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)),
+                    ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)),
+                    new RetryNTimes(ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)),
+                            ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL))));
             _curator.start();
         } catch (Exception ex) {
             LOG.error("Couldn't connect to zookeeper", ex);

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java
index ad96006..9f26479 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java
@@ -18,7 +18,7 @@
 package org.apache.storm.kafka;
 
 import org.apache.storm.Config;
-import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryNTimes;
@@ -43,10 +43,10 @@ public class ZkState {
             serverPorts = serverPorts + server + ":" + port + ",";
         }
         return CuratorFrameworkFactory.newClient(serverPorts,
-                Utils.getInt(stateConf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)),
-                Utils.getInt(stateConf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)),
-                new RetryNTimes(Utils.getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)),
-                        Utils.getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL))));
+                ObjectReader.getInt(stateConf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)),
+                ObjectReader.getInt(stateConf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)),
+                new RetryNTimes(ObjectReader.getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)),
+                        ObjectReader.getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL))));
     }
 
     public CuratorFramework getCurator() {

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java
index 9da6c0a..7fd1d41 100644
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java
+++ b/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java
@@ -33,6 +33,7 @@ import kafka.message.MessageAndOffset;
 
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.storm.utils.Utils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -43,7 +44,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.storm.kafka.trident.GlobalPartitionInformation;
 import org.apache.storm.spout.SchemeAsMultiScheme;
-import org.apache.storm.utils.Utils;
 
 import com.google.common.collect.ImmutableMap;
 public class KafkaUtilsTest {

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java b/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java
index cc3f2be..c7ba674 100644
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java
+++ b/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java
@@ -17,14 +17,12 @@
  */
 package org.apache.storm.kafka;
 
-import org.apache.storm.Config;
 import org.apache.storm.utils.Utils;
 import kafka.api.OffsetRequest;
 import kafka.javaapi.consumer.SimpleConsumer;
 import kafka.javaapi.message.ByteBufferMessageSet;
 import kafka.message.Message;
 import kafka.message.MessageAndOffset;
-import org.apache.storm.kafka.bolt.KafkaBolt;
 import org.apache.storm.kafka.trident.GlobalPartitionInformation;
 
 import java.nio.ByteBuffer;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
index 3b8bea1..cebc261 100644
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
+++ b/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
@@ -27,8 +27,8 @@ import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.TupleImpl;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.TupleUtils;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.TupleUtils;
 import com.google.common.collect.ImmutableList;
 import kafka.api.OffsetRequest;
 import kafka.api.FetchRequest;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kinesis/README.md
----------------------------------------------------------------------
diff --git a/external/storm-kinesis/README.md b/external/storm-kinesis/README.md
index f163a54..8eaf532 100644
--- a/external/storm-kinesis/README.md
+++ b/external/storm-kinesis/README.md
@@ -106,7 +106,7 @@ Aws sdk version that this was tested with is 1.10.77
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-kinesis/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kinesis/pom.xml b/external/storm-kinesis/pom.xml
index 763d9c2..fb0b527 100644
--- a/external/storm-kinesis/pom.xml
+++ b/external/storm-kinesis/pom.xml
@@ -40,7 +40,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-metrics/pom.xml b/external/storm-metrics/pom.xml
index ef51c56..48a5cc0 100644
--- a/external/storm-metrics/pom.xml
+++ b/external/storm-metrics/pom.xml
@@ -57,7 +57,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.storm</groupId>
-      <artifactId>storm-core</artifactId>
+      <artifactId>storm-client</artifactId>
       <version>${project.version}</version>
       <!-- keep storm out of the jar-with-dependencies -->
       <scope>${provided.scope}</scope>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-mongodb/pom.xml b/external/storm-mongodb/pom.xml
index 47b6f63..6aaeab6 100644
--- a/external/storm-mongodb/pom.xml
+++ b/external/storm-mongodb/pom.xml
@@ -42,9 +42,9 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
-            <scope>${provided.scope}</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.mongodb</groupId>
@@ -71,4 +71,4 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-mqtt/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-mqtt/pom.xml b/external/storm-mqtt/pom.xml
index 37c19d7..0084f4f 100644
--- a/external/storm-mqtt/pom.xml
+++ b/external/storm-mqtt/pom.xml
@@ -66,11 +66,17 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-server</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.fusesource.mqtt-client</groupId>
             <artifactId>mqtt-client</artifactId>
             <version>1.10</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-opentsdb/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-opentsdb/pom.xml b/external/storm-opentsdb/pom.xml
index 41f7e55..c5b35b4 100644
--- a/external/storm-opentsdb/pom.xml
+++ b/external/storm-opentsdb/pom.xml
@@ -42,7 +42,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-pmml/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-pmml/pom.xml b/external/storm-pmml/pom.xml
index b44c6f5..f6e26ba 100644
--- a/external/storm-pmml/pom.xml
+++ b/external/storm-pmml/pom.xml
@@ -49,7 +49,7 @@
         <!--parent module dependency-->
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-redis/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-redis/pom.xml b/external/storm-redis/pom.xml
index 102e0f5..adc90b2 100644
--- a/external/storm-redis/pom.xml
+++ b/external/storm-redis/pom.xml
@@ -47,7 +47,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-solr/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-solr/pom.xml b/external/storm-solr/pom.xml
index 14922ed..057ab5b 100644
--- a/external/storm-solr/pom.xml
+++ b/external/storm-solr/pom.xml
@@ -40,7 +40,7 @@
     <dependencies>
        <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/flux/README.md
----------------------------------------------------------------------
diff --git a/flux/README.md b/flux/README.md
index 206ae22..5aa76ae 100644
--- a/flux/README.md
+++ b/flux/README.md
@@ -113,6 +113,17 @@ The current version of Flux is available in Maven Central at the following coord
     <artifactId>flux-core</artifactId>
     <version>${storm.version}</version>
 </dependency>
+<dependency>
+    <groupId>org.apache.storm</groupId>
+    <artifactId>storm-server</artifactId>
+    <version>${storm.version}</version>
+    <exclustions>
+        <exclusion>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-client</artifactId>
+        </exclusion>
+    </exclustions>
+</dependency>
 ```
 
 Using shell spouts and bolts requires additional Flux Wrappers library:
@@ -136,6 +147,17 @@ The example below illustrates Flux usage with the Maven shade plugin:
         <artifactId>flux-core</artifactId>
         <version>${storm.version}</version>
     </dependency>
+    <dependency>
+        <groupId>org.apache.storm</groupId>
+        <artifactId>storm-server</artifactId>
+        <version>${storm.version}</version>
+        <exclustions>
+            <exclusion>
+                <groupId>org.apache.storm</groupId>
+                <artifactId>storm-client</artifactId>
+            </exclusion>
+        </exclustions>
+    </dependency>
     <!-- Flux Wrappers include -->
     <dependency>
         <groupId>org.apache.storm</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
index 7093105..5599561 100644
--- a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
@@ -19,7 +19,6 @@ package org.apache.storm.flux.examples;
 
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.IBasicBolt;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.base.BaseBasicBolt;
 import org.apache.storm.tuple.Fields;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/flux/pom.xml
----------------------------------------------------------------------
diff --git a/flux/pom.xml b/flux/pom.xml
index ce8f6a0..d1f330f 100644
--- a/flux/pom.xml
+++ b/flux/pom.xml
@@ -44,9 +44,16 @@
     </modules>
 
     <dependencies>
+        <!--
+        Since Flux uses LocalCluster to provide the feature: running topology locally...
+        User should notice that configured topology will be run with 'storm-client' dependencies
+        when adding Flux into topology dependency.
+        If user want to run topology with 'storm-server' dependencies, user can just include
+        'storm-server' as 'compile' scope, and exclude 'storm-client' from 'storm-server'.
+        -->
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/integration-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration-test/pom.xml b/integration-test/pom.xml
index a1081ac..413eb3c 100755
--- a/integration-test/pom.xml
+++ b/integration-test/pom.xml
@@ -78,7 +78,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java
----------------------------------------------------------------------
diff --git a/integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java b/integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java
index c4200a0..5cef9fa 100644
--- a/integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java
+++ b/integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java
@@ -20,7 +20,7 @@ package org.apache.storm.st.tests.window;
 import org.apache.storm.st.helper.AbstractTest;
 import org.apache.storm.st.wrapper.LogData;
 import org.apache.storm.st.wrapper.TopoWrap;
-import org.apache.storm.thrift.TException;
+import org.apache.thrift.TException;
 import org.apache.storm.st.topology.TestableTopology;
 import org.apache.storm.st.topology.window.SlidingTimeCorrectness;
 import org.apache.storm.st.topology.window.SlidingWindowCorrectness;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
----------------------------------------------------------------------
diff --git a/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
index 7311d5b..f97a259 100644
--- a/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
+++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
@@ -26,7 +26,7 @@ import org.apache.storm.generated.TopologyInfo;
 import org.apache.storm.generated.TopologySummary;
 import org.apache.storm.st.utils.AssertUtil;
 import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.storm.thrift.TException;
+import org.apache.thrift.TException;
 import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java
----------------------------------------------------------------------
diff --git a/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java
index 2be0402..ac6d0c7 100644
--- a/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java
+++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java
@@ -39,7 +39,7 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.thrift.TException;
+import org.apache.thrift.TException;
 import org.apache.storm.st.topology.window.data.FromJson;
 import org.apache.storm.st.utils.StringDecorator;
 import org.apache.storm.st.utils.TimeUtil;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5a3a61b..1281171 100644
--- a/pom.xml
+++ b/pom.xml
@@ -319,8 +319,10 @@
         <module>storm-multilang/ruby</module>
         <module>storm-buildtools/maven-shade-clojure-transformer</module>
         <module>storm-buildtools/storm-maven-plugins</module>
+        <module>storm-client</module>
+        <module>storm-server</module>
         <module>storm-core</module>
-        <module>storm-drpc-server</module>
+        <module>storm-webapp</module>
         <module>storm-rename-hack</module>
         <module>storm-clojure</module>
         <module>storm-submit-tools</module>
@@ -366,6 +368,7 @@
         <module>examples/storm-pmml-examples</module>
         <module>examples/storm-jms-examples</module>
         <module>examples/storm-perf</module>
+        <module>storm-client-misc</module>
     </modules>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/sql/storm-sql-core/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/pom.xml b/sql/storm-sql-core/pom.xml
index e409658..81d2139 100644
--- a/sql/storm-sql-core/pom.xml
+++ b/sql/storm-sql-core/pom.xml
@@ -38,12 +38,18 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
+            <artifactId>storm-server</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
             <artifactId>storm-sql-runtime</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/sql/storm-sql-external/storm-sql-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/pom.xml b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
index d1f37b7..2a0eae5 100644
--- a/sql/storm-sql-external/storm-sql-hdfs/pom.xml
+++ b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
@@ -38,7 +38,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
             <exclusions>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/sql/storm-sql-external/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/pom.xml b/sql/storm-sql-external/storm-sql-kafka/pom.xml
index c7562bb..a5d4131 100644
--- a/sql/storm-sql-external/storm-sql-kafka/pom.xml
+++ b/sql/storm-sql-external/storm-sql-kafka/pom.xml
@@ -38,7 +38,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/sql/storm-sql-external/storm-sql-mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/pom.xml b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
index cb0baba..ae3c715 100644
--- a/sql/storm-sql-external/storm-sql-mongodb/pom.xml
+++ b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
@@ -38,7 +38,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/sql/storm-sql-external/storm-sql-redis/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/pom.xml b/sql/storm-sql-external/storm-sql-redis/pom.xml
index 38648a4..34082e4 100644
--- a/sql/storm-sql-external/storm-sql-redis/pom.xml
+++ b/sql/storm-sql-external/storm-sql-redis/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/sql/storm-sql-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/pom.xml b/sql/storm-sql-runtime/pom.xml
index 060a1ab..065ed72 100644
--- a/sql/storm-sql-runtime/pom.xml
+++ b/sql/storm-sql-runtime/pom.xml
@@ -38,7 +38,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
index 6b3dfc9..2dcd66c 100644
--- a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
@@ -32,7 +32,7 @@ import org.apache.storm.sql.runtime.serde.json.JsonScheme;
 import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
 import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
 import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
-import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ReflectionUtils;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -56,7 +56,7 @@ public final class SerdeUtils {
                 Preconditions.checkArgument(isNotEmpty(schemaString), "input.avro.schema can not be empty");
                 scheme = new AvroScheme(schemaString, fieldNames);
             } else {
-                scheme = Utils.newInstance(inputFormatClass);
+                scheme = ReflectionUtils.newInstance(inputFormatClass);
             }
         } else {
             //use JsonScheme as the default scheme
@@ -80,7 +80,7 @@ public final class SerdeUtils {
                 Preconditions.checkArgument(isNotEmpty(schemaString), "output.avro.schema can not be empty");
                 serializer = new AvroSerializer(schemaString, fieldNames);
             } else {
-                serializer = Utils.newInstance(outputFormatClass);
+                serializer = ReflectionUtils.newInstance(outputFormatClass);
             }
         } else {
             //use JsonSerializer as the default serializer

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client-misc/pom.xml
----------------------------------------------------------------------
diff --git a/storm-client-misc/pom.xml b/storm-client-misc/pom.xml
new file mode 100644
index 0000000..f4f46c9
--- /dev/null
+++ b/storm-client-misc/pom.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~ http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>storm</artifactId>
+        <groupId>org.apache.storm</groupId>
+        <version>2.0.0-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>storm-client-misc</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-client</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java
----------------------------------------------------------------------
diff --git a/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java b/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java
new file mode 100644
index 0000000..fd732da
--- /dev/null
+++ b/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.misc.metric;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.net.URL;
+import java.net.HttpURLConnection;
+
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.storm.serialization.KryoValuesSerializer;
+
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.task.IErrorReporter;
+import org.apache.storm.task.TopologyContext;
+
+/**
+ * Listens for all metrics and POSTs them serialized to a configured URL
+ *
+ * To use, add this to your topology's configuration:
+ *
+ * ```java
+ *   conf.registerMetricsConsumer(org.apache.storm.metrics.HttpForwardingMetricsConsumer.class, "http://example.com:8080/metrics/my-topology/", 1);
+ * ```
+ *
+ * The body of the post is data serialized using {@link org.apache.storm.serialization.KryoValuesSerializer}, with the data passed in
+ * as a list of `[TaskInfo, Collection<DataPoint>]`.  More things may be appended to the end of the list in the future.
+ *
+ * The values can be deserialized using the org.apache.storm.serialization.KryoValuesDeserializer, and a 
+ * correct config + classpath.
+ *
+ * @see org.apache.storm.serialization.KryoValuesSerializer
+ */
+public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
+    private transient URL _url; 
+    private transient IErrorReporter _errorReporter;
+    private transient KryoValuesSerializer _serializer;
+
+    @Override
+    public void prepare(Map stormConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { 
+        try {
+            _url = new URL((String)registrationArgument);
+            _errorReporter = errorReporter;
+            _serializer = new KryoValuesSerializer(stormConf);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void handleDataPoints(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
+        try {
+            HttpURLConnection con = (HttpURLConnection)_url.openConnection();
+            con.setRequestMethod("POST");
+            con.setDoOutput(true);
+            Output out = new Output(con.getOutputStream());
+            _serializer.serializeInto(Arrays.asList(taskInfo, dataPoints), out);
+            out.flush();
+            out.close();
+            //The connection is not sent unless a response is requested
+            int response = con.getResponseCode();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void cleanup() { }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java
----------------------------------------------------------------------
diff --git a/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java b/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java
new file mode 100644
index 0000000..783d0ba
--- /dev/null
+++ b/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.misc.metric;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.List;
+import java.net.ServerSocket;
+import java.net.InetAddress;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.ServletException;
+
+import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
+import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
+
+import com.esotericsoftware.kryo.io.Input;
+import org.apache.storm.serialization.KryoValuesDeserializer;
+import org.apache.storm.utils.Utils;
+
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+/**
+ * A server that can listen for metrics from the HttpForwardingMetricsConsumer.
+ */
+public abstract class HttpForwardingMetricsServer {
+    private Map _conf;
+    private Server _server = null;
+    private int _port = -1;
+    private String _url = null;
+
+    ThreadLocal<KryoValuesDeserializer> _des = new ThreadLocal<KryoValuesDeserializer>() {
+        @Override
+        protected KryoValuesDeserializer initialValue() {
+            return new KryoValuesDeserializer(_conf);
+        }
+    };
+
+    private class MetricsCollectionServlet extends HttpServlet
+    {
+        protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException
+        {
+            Input in = new Input(request.getInputStream());
+            List<Object> metrics = _des.get().deserializeFrom(in);
+            handle((TaskInfo)metrics.get(0), (Collection<DataPoint>)metrics.get(1));
+            response.setStatus(HttpServletResponse.SC_OK);
+        }
+    }
+
+    public HttpForwardingMetricsServer(Map conf) {
+        _conf = Utils.readStormConfig();
+        if (conf != null) {
+            _conf.putAll(conf);
+        }
+    }
+
+    //This needs to be thread safe
+    public abstract void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints);
+
+    public void serve(Integer port) {
+        try {
+            if (_server != null) throw new RuntimeException("The server is already running");
+    
+            if (port == null || port <= 0) {
+                ServerSocket s = new ServerSocket(0);
+                port = s.getLocalPort();
+                s.close();
+            }
+            _server = new Server(port);
+            _port = port;
+            _url = "http://"+InetAddress.getLocalHost().getHostName()+":"+_port+"/";
+ 
+            ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+            context.setContextPath("/");
+            _server.setHandler(context);
+ 
+            context.addServlet(new ServletHolder(new MetricsCollectionServlet()),"/*");
+
+            _server.start();
+         } catch (RuntimeException e) {
+             throw e;
+         } catch (Exception e) {
+             throw new RuntimeException(e);
+         }
+    }
+
+    public void serve() {
+        serve(null);
+    }
+
+    public int getPort() {
+        return _port;
+    }
+
+    public String getUrl() {
+        return _url;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/pom.xml
----------------------------------------------------------------------
diff --git a/storm-client/pom.xml b/storm-client/pom.xml
new file mode 100644
index 0000000..9813593
--- /dev/null
+++ b/storm-client/pom.xml
@@ -0,0 +1,336 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~ http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>storm</artifactId>
+        <groupId>org.apache.storm</groupId>
+        <version>2.0.0-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>storm-client</artifactId>
+    <packaging>jar</packaging>
+    <name>Storm Client</name>
+    <description>The client side (including worker) of Storm implementations.</description>
+
+    <dependencies>
+        <!-- logging -->
+
+        <!--Hadoop Mini Cluster cannot use log4j2 bridge,
+    Surefire has a way to exclude the conflicting log4j API jar
+    from the classpath, classpathDependencyExcludes, but it didn't work in practice.
+    This is here as a work around to place it at the beginning of the classpath
+    even though maven does not officially support ordering of the classpath.-->
+        <dependency>
+            <groupId>uk.org.lidalia</groupId>
+            <artifactId>sysout-over-slf4j</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-slf4j-impl</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+        </dependency>
+
+        <!-- guava -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <!-- thrift -->
+        <dependency>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!-- commons -->
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>commons-collections</groupId>
+            <artifactId>commons-collections</artifactId>
+        </dependency>
+
+        <!-- disruptor -->
+        <dependency>
+            <groupId>com.lmax</groupId>
+            <artifactId>disruptor</artifactId>
+        </dependency>
+
+        <!-- json -->
+        <dependency>
+            <groupId>com.googlecode.json-simple</groupId>
+            <artifactId>json-simple</artifactId>
+        </dependency>
+
+        <!-- yaml -->
+        <dependency>
+            <groupId>org.yaml</groupId>
+            <artifactId>snakeyaml</artifactId>
+        </dependency>
+
+        <!-- clojure -->
+        <dependency>
+            <groupId>org.clojure</groupId>
+            <artifactId>clojure</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.twitter</groupId>
+            <artifactId>carbonite</artifactId>
+            <scope>compile</scope>
+        </dependency>
+
+        <!-- netty -->
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+        </dependency>
+
+        <!-- kyro -->
+        <dependency>
+            <groupId>com.esotericsoftware</groupId>
+            <artifactId>kryo</artifactId>
+        </dependency>
+
+        <!-- zookeeper -->
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+        </dependency>
+
+        <!-- curator -->
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+        </dependency>
+
+        <!-- jgrapht -->
+        <dependency>
+            <groupId>org.jgrapht</groupId>
+            <artifactId>jgrapht-core</artifactId>
+            <scope>compile</scope>
+        </dependency>
+
+        <!-- servlet -->
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>servlet-api</artifactId>
+        </dependency>
+
+        <!-- belows are transitive dependencies which are version managed in storm pom -->
+        <!-- httpclient is derived from thrift -->
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+        </dependency>
+
+        <!-- curator-client is derived from curator-framework -->
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-client</artifactId>
+        </dependency>
+
+        <!-- end of transitive dependency management -->
+
+        <!-- test -->
+        <!-- hamcrest-core dependency is shaded inside the mockito-all and junit depends on newer version of hamcrest-core.
+        To give higher precedence to classes from newer version of hamcrest-core, Junit has been placed above mockito.
+         -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <sourceDirectory>src/jvm</sourceDirectory>
+        <testSourceDirectory>test/jvm</testSourceDirectory>
+        <resources>
+            <resource>
+                <directory>../conf</directory>
+            </resource>
+            <resource>
+                <directory>../</directory>
+                <targetPath>META-INF</targetPath>
+                <includes>
+                    <include>NOTICE</include>
+                </includes>
+            </resource>
+            <resource>
+                <directory>${basedir}/src/resources</directory>
+                <excludes>
+                    <exclude>storm-client-version-info.properties</exclude>
+                </excludes>
+                <filtering>false</filtering>
+            </resource>
+            <resource>
+                <directory>${basedir}/src/resources</directory>
+                <includes>
+                    <include>storm-client-version-info.properties</include>
+                </includes>
+                <filtering>true</filtering>
+            </resource>
+        </resources>
+        <testResources>
+            <testResource>
+                <directory>src/dev</directory>
+            </testResource>
+            <testResource>
+                <directory>test/resources</directory>
+            </testResource>
+        </testResources>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-report-plugin</artifactId>
+                <configuration>
+                    <reportsDirectories>
+                        <file>${project.build.directory}/test-reports</file>
+                    </reportsDirectories>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-failsafe-plugin</artifactId>
+                <configuration>
+                    <reportsDirectory>${project.build.directory}/test-reports</reportsDirectory>
+                </configuration>
+            </plugin>
+            <plugin>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.8</version>
+                <executions>
+                    <execution>
+                        <id>copy-dependencies</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <overWriteReleases>false</overWriteReleases>
+                            <overWriteSnapshots>false</overWriteSnapshots>
+                            <overWriteIfNewer>true</overWriteIfNewer>
+                            <includeScope>runtime</includeScope>
+                        </configuration>
+                    </execution>
+                    <!-- multi-lang resources -->
+                    <execution>
+                        <id>unpack</id>
+                        <phase>process-test-resources</phase>
+                        <goals>
+                            <goal>unpack</goal>
+                        </goals>
+                        <configuration>
+                            <artifactItems>
+                                <artifactItem>
+                                    <groupId>org.apache.storm</groupId>
+                                    <artifactId>multilang-ruby</artifactId>
+                                    <version>${project.version}</version>
+                                </artifactItem>
+                                <artifactItem>
+                                    <groupId>org.apache.storm</groupId>
+                                    <artifactId>multilang-python</artifactId>
+                                    <version>${project.version}</version>
+                                </artifactItem>
+                                <artifactItem>
+                                    <groupId>org.apache.storm</groupId>
+                                    <artifactId>multilang-javascript</artifactId>
+                                    <version>${project.version}</version>
+                                </artifactItem>
+                            </artifactItems>
+                            <outputDirectory>${project.build.directory}/test-classes</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>2.2.1</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <goals>
+                            <!-- avoid warning about recursion -->
+                            <goal>jar-no-fork</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.6</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/genthrift.sh
----------------------------------------------------------------------
diff --git a/storm-client/src/genthrift.sh b/storm-client/src/genthrift.sh
new file mode 100644
index 0000000..a2373c2
--- /dev/null
+++ b/storm-client/src/genthrift.sh
@@ -0,0 +1,34 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+rm -rf gen-javabean gen-py py
+rm -rf jvm/org/apache/storm/generated
+thrift --gen java:beans,hashcode,nocamel,generated_annotations=undated --gen py:utf8strings storm.thrift
+for file in gen-javabean/org/apache/storm/generated/* ; do
+  cat java_license_header.txt ${file} > ${file}.tmp
+  mv -f ${file}.tmp ${file}
+done
+cat py_license_header.txt gen-py/__init__.py > gen-py/__init__.py.tmp
+mv gen-py/__init__.py.tmp gen-py/__init__.py
+for file in gen-py/storm/* ; do
+  cat py_license_header.txt ${file} > ${file}.tmp
+  mv -f ${file}.tmp ${file}
+done
+mkdir -p jvm/org/apache/storm
+mv gen-javabean/org/apache/storm/generated jvm/org/apache/storm/generated
+mv gen-py py
+rm -rf gen-javabean

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/java_license_header.txt
----------------------------------------------------------------------
diff --git a/storm-client/src/java_license_header.txt b/storm-client/src/java_license_header.txt
new file mode 100644
index 0000000..7e66353
--- /dev/null
+++ b/storm-client/src/java_license_header.txt
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */


[49/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java
new file mode 100644
index 0000000..0e9b6ea
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.starter.bolt.PrinterBolt;
+import org.apache.storm.starter.spout.RandomIntegerSpout;
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.state.State;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseStatefulWindowedBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.windowing.TupleWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
+
+/**
+ * A simple example that demonstrates the usage of {@link org.apache.storm.topology.IStatefulWindowedBolt} to
+ * save the state of the windowing operation to avoid re-computation in case of failures.
+ * <p>
+ * The framework internally manages the window boundaries and does not invoke
+ * {@link org.apache.storm.topology.IWindowedBolt#execute(TupleWindow)} for the already evaluated windows in case of restarts
+ * during failures. The {@link org.apache.storm.topology.IStatefulBolt#initState(State)}
+ * is invoked with the previously saved state of the bolt after prepare, before the execute() method is invoked.
+ * </p>
+ */
+public class StatefulWindowingTopology {
+    private static final Logger LOG = LoggerFactory.getLogger(StatefulWindowingTopology.class);
+
+    private static class WindowSumBolt extends BaseStatefulWindowedBolt<KeyValueState<String, Long>> {
+        private KeyValueState<String, Long> state;
+        private long sum;
+
+        private OutputCollector collector;
+
+        @Override
+        public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+            this.collector = collector;
+        }
+
+        @Override
+        public void initState(KeyValueState<String, Long> state) {
+            this.state = state;
+            sum = state.get("sum", 0L);
+            LOG.debug("initState with state [" + state + "] current sum [" + sum + "]");
+        }
+
+        @Override
+        public void execute(TupleWindow inputWindow) {
+            for (Tuple tuple : inputWindow.get()) {
+                sum += tuple.getIntegerByField("value");
+            }
+            state.put("sum", sum);
+            collector.emit(new Values(sum));
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("sum"));
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        TopologyBuilder builder = new TopologyBuilder();
+        builder.setSpout("spout", new RandomIntegerSpout());
+        builder.setBolt("sumbolt", new WindowSumBolt().withWindow(new Count(5), new Count(3))
+                .withMessageIdField("msgid"), 1).shuffleGrouping("spout");
+        builder.setBolt("printer", new PrinterBolt(), 1).shuffleGrouping("sumbolt");
+        Config conf = new Config();
+        conf.setDebug(false);
+        //conf.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
+        if (args != null && args.length > 0) {
+            conf.setNumWorkers(1);
+            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+        } else {
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalTopology topo = cluster.submitTopology("test", conf, builder.createTopology());) {
+                Utils.sleep(40000);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
index 9e709a1..d8137b0 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
@@ -20,16 +20,13 @@ package org.apache.storm.starter;
 import org.apache.storm.Config;
 import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.metric.HttpForwardingMetricsServer;
-import org.apache.storm.metric.HttpForwardingMetricsConsumer;
-import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.misc.metric.HttpForwardingMetricsServer;
 import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
 import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
 import org.apache.storm.generated.*;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.IRichBolt;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseBasicBolt;
@@ -37,9 +34,8 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
-import org.apache.storm.StormSubmitter;
+import org.apache.storm.utils.NimbusClient;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -397,7 +393,7 @@ public class ThroughputVsLatency {
     C cluster = new C(conf);
     conf.setNumWorkers(parallelism);
     conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class);
-    conf.registerMetricsConsumer(org.apache.storm.metric.HttpForwardingMetricsConsumer.class, url, 1);
+    conf.registerMetricsConsumer(org.apache.storm.misc.metric.HttpForwardingMetricsConsumer.class, url, 1);
     Map<String, String> workerMetrics = new HashMap<String, String>();
     if (!cluster.isLocal()) {
       //sigar uses JNI and does not work in local mode

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
index 7467634..cbc5d45 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
@@ -22,21 +22,13 @@ import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.starter.spout.RandomIntegerSpout;
 import org.apache.storm.streams.Pair;
-import org.apache.storm.streams.PairStream;
-import org.apache.storm.streams.Stream;
 import org.apache.storm.streams.StreamBuilder;
 import org.apache.storm.streams.operations.CombinerAggregator;
-import org.apache.storm.streams.operations.mappers.TupleValueMapper;
-import org.apache.storm.streams.operations.mappers.TupleValueMappers;
 import org.apache.storm.streams.operations.mappers.ValueMapper;
-import org.apache.storm.streams.tuple.Tuple3;
 import org.apache.storm.streams.windowing.TumblingWindows;
 import org.apache.storm.topology.base.BaseWindowedBolt;
-import org.apache.storm.trident.windowing.config.TumblingDurationWindow;
 import org.apache.storm.utils.Utils;
 
-import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
-
 /**
  * An example that illustrates the global aggregate
  */

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java b/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
deleted file mode 100644
index d073350..0000000
--- a/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package storm.starter;
-
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.starter.spout.RandomIntegerSpout;
-import org.apache.storm.state.KeyValueState;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.topology.base.BaseStatefulBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-/**
- * An example topology that demonstrates the use of {@link org.apache.storm.topology.IStatefulBolt}
- * to manage state. To run the example,
- * <pre>
- * $ storm jar examples/storm-starter/storm-starter-topologies-*.jar storm.starter.StatefulTopology statetopology
- * </pre>
- * <p/>
- * The default state used is 'InMemoryKeyValueState' which does not persist the state across restarts. You could use
- * 'RedisKeyValueState' to test state persistence by setting below property in conf/storm.yaml
- * <pre>
- * topology.state.provider: org.apache.storm.redis.state.RedisKeyValueStateProvider
- * </pre>
- * <p/>
- * You should also start a local redis instance before running the 'storm jar' command. The default
- * RedisKeyValueStateProvider parameters can be overridden in conf/storm.yaml, for e.g.
- * <p/>
- * <pre>
- * topology.state.provider.config: '{"keyClass":"...", "valueClass":"...",
- *                                   "keySerializerClass":"...", "valueSerializerClass":"...",
- *                                   "jedisPoolConfig":{"host":"localhost", "port":6379,
- *                                      "timeout":2000, "database":0, "password":"xyz"}}'
- *
- * </pre>
- * </p>
- */
-public class StatefulTopology {
-    private static final Logger LOG = LoggerFactory.getLogger(StatefulTopology.class);
-    /**
-     * A bolt that uses {@link KeyValueState} to save its state.
-     */
-    private static class StatefulSumBolt extends BaseStatefulBolt<KeyValueState<String, Long>> {
-        String name;
-        KeyValueState<String, Long> kvState;
-        long sum;
-        private OutputCollector collector;
-
-        StatefulSumBolt(String name) {
-            this.name = name;
-        }
-
-        @Override
-        public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-            this.collector = collector;
-        }
-
-        @Override
-        public void execute(Tuple input) {
-            sum += ((Number) input.getValueByField("value")).longValue();
-            LOG.debug("{} sum = {}", name, sum);
-            kvState.put("sum", sum);
-            collector.emit(input, new Values(sum));
-            collector.ack(input);
-        }
-
-        @Override
-        public void initState(KeyValueState<String, Long> state) {
-            kvState = state;
-            sum = kvState.get("sum", 0L);
-            LOG.debug("Initstate, sum from saved state = {} ", sum);
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("value"));
-        }
-    }
-
-    public static class PrinterBolt extends BaseBasicBolt {
-        @Override
-        public void execute(Tuple tuple, BasicOutputCollector collector) {
-            System.out.println(tuple);
-            LOG.debug("Got tuple {}", tuple);
-            collector.emit(tuple.getValues());
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer ofd) {
-            ofd.declare(new Fields("value"));
-        }
-
-    }
-
-    public static void main(String[] args) throws Exception {
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new RandomIntegerSpout());
-        builder.setBolt("partialsum", new StatefulSumBolt("partial"), 1).shuffleGrouping("spout");
-        builder.setBolt("printer", new PrinterBolt(), 2).shuffleGrouping("partialsum");
-        builder.setBolt("total", new StatefulSumBolt("total"), 1).shuffleGrouping("printer");
-        Config conf = new Config();
-        conf.setDebug(false);
-
-        if (args != null && args.length > 0) {
-            conf.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topology = cluster.submitTopology("test", conf, builder.createTopology());) {
-                Utils.sleep(40000);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/storm/starter/StatefulWindowingTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/StatefulWindowingTopology.java b/examples/storm-starter/src/jvm/storm/starter/StatefulWindowingTopology.java
deleted file mode 100644
index e43759c..0000000
--- a/examples/storm-starter/src/jvm/storm/starter/StatefulWindowingTopology.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package storm.starter;
-
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.starter.bolt.PrinterBolt;
-import org.apache.storm.starter.spout.RandomIntegerSpout;
-import org.apache.storm.state.KeyValueState;
-import org.apache.storm.state.State;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.topology.base.BaseStatefulWindowedBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.windowing.TupleWindow;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
-
-/**
- * A simple example that demonstrates the usage of {@link org.apache.storm.topology.IStatefulWindowedBolt} to
- * save the state of the windowing operation to avoid re-computation in case of failures.
- * <p>
- * The framework internally manages the window boundaries and does not invoke
- * {@link org.apache.storm.topology.IWindowedBolt#execute(TupleWindow)} for the already evaluated windows in case of restarts
- * during failures. The {@link org.apache.storm.topology.IStatefulBolt#initState(State)}
- * is invoked with the previously saved state of the bolt after prepare, before the execute() method is invoked.
- * </p>
- */
-public class StatefulWindowingTopology {
-    private static final Logger LOG = LoggerFactory.getLogger(StatefulWindowingTopology.class);
-
-    private static class WindowSumBolt extends BaseStatefulWindowedBolt<KeyValueState<String, Long>> {
-        private KeyValueState<String, Long> state;
-        private long sum;
-
-        private OutputCollector collector;
-
-        @Override
-        public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-            this.collector = collector;
-        }
-
-        @Override
-        public void initState(KeyValueState<String, Long> state) {
-            this.state = state;
-            sum = state.get("sum", 0L);
-            LOG.debug("initState with state [" + state + "] current sum [" + sum + "]");
-        }
-
-        @Override
-        public void execute(TupleWindow inputWindow) {
-            for (Tuple tuple : inputWindow.get()) {
-                sum += tuple.getIntegerByField("value");
-            }
-            state.put("sum", sum);
-            collector.emit(new Values(sum));
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("sum"));
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new RandomIntegerSpout());
-        builder.setBolt("sumbolt", new WindowSumBolt().withWindow(new Count(5), new Count(3))
-                .withMessageIdField("msgid"), 1).shuffleGrouping("spout");
-        builder.setBolt("printer", new PrinterBolt(), 1).shuffleGrouping("sumbolt");
-        Config conf = new Config();
-        conf.setDebug(false);
-        //conf.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
-        if (args != null && args.length > 0) {
-            conf.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", conf, builder.createTopology());) {
-                Utils.sleep(40000);
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-cassandra/pom.xml b/external/storm-cassandra/pom.xml
index 8fc882a..c4e341b 100644
--- a/external/storm-cassandra/pom.xml
+++ b/external/storm-cassandra/pom.xml
@@ -79,12 +79,19 @@
 
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>
 
         <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-server</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
             <version>${org.slf4j.version}</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/bolt/BaseCassandraBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/bolt/BaseCassandraBolt.java b/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/bolt/BaseCassandraBolt.java
index ea7750f..7d34b16 100644
--- a/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/bolt/BaseCassandraBolt.java
+++ b/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/bolt/BaseCassandraBolt.java
@@ -35,11 +35,9 @@ import org.apache.storm.cassandra.query.CQLStatementTupleMapper;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseRichBolt;
 import org.apache.storm.topology.base.BaseTickTupleAwareRichBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
-import org.apache.storm.utils.TupleUtils;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/client/CassandraConf.java
----------------------------------------------------------------------
diff --git a/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/client/CassandraConf.java b/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/client/CassandraConf.java
index 0f0de53..28e47bd 100644
--- a/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/client/CassandraConf.java
+++ b/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/client/CassandraConf.java
@@ -19,6 +19,7 @@
 package org.apache.storm.cassandra.client;
 
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
 import com.datastax.driver.core.ConsistencyLevel;
 import com.datastax.driver.core.policies.DefaultRetryPolicy;
 import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy;
@@ -106,14 +107,14 @@ public class CassandraConf implements Serializable {
      */
     public CassandraConf(Map<String, Object> conf) {
 
-        this.username = (String)Utils.get(conf, CASSANDRA_USERNAME, null);
-        this.password = (String)Utils.get(conf, CASSANDRA_PASSWORD, null);
+        this.username = (String) Utils.get(conf, CASSANDRA_USERNAME, null);
+        this.password = (String) Utils.get(conf, CASSANDRA_PASSWORD, null);
         this.keyspace = get(conf, CASSANDRA_KEYSPACE);
-        this.consistencyLevel = ConsistencyLevel.valueOf((String)Utils.get(conf, CASSANDRA_CONSISTENCY_LEVEL, ConsistencyLevel.ONE.name()));
-        this.nodes    = ((String)Utils.get(conf, CASSANDRA_NODES, "localhost")).split(",");
-        this.batchSizeRows = Utils.getInt(conf.get(CASSANDRA_BATCH_SIZE_ROWS), 100);
-        this.port = Utils.getInt(conf.get(CASSANDRA_PORT), 9042);
-        this.retryPolicyName = (String)Utils.get(conf, CASSANDRA_RETRY_POLICY, DefaultRetryPolicy.class.getSimpleName());
+        this.consistencyLevel = ConsistencyLevel.valueOf((String) Utils.get(conf, CASSANDRA_CONSISTENCY_LEVEL, ConsistencyLevel.ONE.name()));
+        this.nodes    = ((String) Utils.get(conf, CASSANDRA_NODES, "localhost")).split(",");
+        this.batchSizeRows = ObjectReader.getInt(conf.get(CASSANDRA_BATCH_SIZE_ROWS), 100);
+        this.port = ObjectReader.getInt(conf.get(CASSANDRA_PORT), 9042);
+        this.retryPolicyName = (String) Utils.get(conf, CASSANDRA_RETRY_POLICY, DefaultRetryPolicy.class.getSimpleName());
         this.reconnectionPolicyBaseMs = getLong(conf.get(CASSANDRA_RECONNECT_POLICY_BASE_MS), 100L);
         this.reconnectionPolicyMaxMs  = getLong(conf.get(CASSANDRA_RECONNECT_POLICY_MAX_MS), TimeUnit.MINUTES.toMillis(1));
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-druid/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-druid/pom.xml b/external/storm-druid/pom.xml
index 3f9601c..6414ccc 100644
--- a/external/storm-druid/pom.xml
+++ b/external/storm-druid/pom.xml
@@ -32,11 +32,17 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-server</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>io.druid</groupId>
             <artifactId>tranquility-core_2.11</artifactId>
             <version>0.8.2</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-elasticsearch/pom.xml b/external/storm-elasticsearch/pom.xml
index c676179..bab0426 100644
--- a/external/storm-elasticsearch/pom.xml
+++ b/external/storm-elasticsearch/pom.xml
@@ -48,7 +48,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/pom.xml b/external/storm-eventhubs/pom.xml
index 05d8c3b..7fe6071 100755
--- a/external/storm-eventhubs/pom.xml
+++ b/external/storm-eventhubs/pom.xml
@@ -58,13 +58,20 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <!-- keep storm out of the jar-with-dependencies -->
             <type>jar</type>
             <scope>${provided.scope}</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-server</artifactId>
+            <version>${project.version}</version>
+            <type>jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.curator</groupId>
             <artifactId>curator-framework</artifactId>
             <version>${curator.version}</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java
deleted file mode 100755
index b0dd33a..0000000
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *******************************************************************************/
-package org.apache.storm.eventhubs.samples;
-
-import java.io.Serializable;
-
-import org.apache.storm.eventhubs.spout.EventHubSpout;
-import org.apache.storm.eventhubs.spout.EventHubSpoutConfig;
-import org.apache.storm.eventhubs.spout.IEventHubReceiver;
-import org.apache.storm.eventhubs.spout.IPartitionManager;
-import org.apache.storm.eventhubs.spout.IPartitionManagerFactory;
-import org.apache.storm.eventhubs.spout.IStateStore;
-import org.apache.storm.eventhubs.spout.SimplePartitionManager;
-
-public class AtMostOnceEventCount extends EventCount implements Serializable {
-  @Override
-  protected EventHubSpout createEventHubSpout() {
-    IPartitionManagerFactory pmFactory = new IPartitionManagerFactory() {
-      private static final long serialVersionUID = 1L;
-
-      @Override
-      public IPartitionManager create(EventHubSpoutConfig spoutConfig,
-          String partitionId, IStateStore stateStore,
-          IEventHubReceiver receiver) {
-        return new SimplePartitionManager(spoutConfig, partitionId,
-            stateStore, receiver);
-      }
-    };
-    EventHubSpout eventHubSpout = new EventHubSpout(
-        spoutConfig, null, pmFactory, null);
-    return eventHubSpout;
-  }
-  
-  public static void main(String[] args) throws Exception {
-    AtMostOnceEventCount scenario = new AtMostOnceEventCount();
-
-    scenario.runScenario(args);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventCount.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventCount.java
deleted file mode 100755
index ae15634..0000000
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventCount.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *******************************************************************************/
-package org.apache.storm.eventhubs.samples;
-
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-
-import org.apache.storm.eventhubs.samples.bolt.GlobalCountBolt;
-import org.apache.storm.eventhubs.samples.bolt.PartialCountBolt;
-import org.apache.storm.eventhubs.spout.EventHubSpout;
-import org.apache.storm.eventhubs.spout.EventHubSpoutConfig;
-
-import java.io.FileReader;
-import java.util.Properties;
-
-/**
- * The basic scenario topology that uses EventHubSpout with PartialCountBolt
- * and GlobalCountBolt.
- * To submit this topology:
- * storm jar {jarfile} {classname} {topologyname} {spoutconffile}
- */
-public class EventCount {
-  protected EventHubSpoutConfig spoutConfig;
-  protected int numWorkers;
-  
-  public EventCount() {
-  }
-  
-  protected void readEHConfig(String[] args) throws Exception {
-	  Properties properties = new Properties();
-    if(args.length > 1) {
-      properties.load(new FileReader(args[1]));
-    }
-    else {
-      properties.load(EventCount.class.getClassLoader().getResourceAsStream(
-          "Config.properties"));
-    }
-
-    String username = properties.getProperty("eventhubspout.username");
-    String password = properties.getProperty("eventhubspout.password");
-    String namespaceName = properties.getProperty("eventhubspout.namespace");
-    String entityPath = properties.getProperty("eventhubspout.entitypath");
-    String targetFqnAddress = properties.getProperty("eventhubspout.targetfqnaddress");
-    String zkEndpointAddress = properties.getProperty("zookeeper.connectionstring");
-    int partitionCount = Integer.parseInt(properties.getProperty("eventhubspout.partitions.count"));
-    int checkpointIntervalInSeconds = Integer.parseInt(properties.getProperty("eventhubspout.checkpoint.interval"));
-    int receiverCredits = Integer.parseInt(properties.getProperty("eventhub.receiver.credits"));
-    String maxPendingMsgsPerPartitionStr = properties.getProperty("eventhubspout.max.pending.messages.per.partition");
-    if(maxPendingMsgsPerPartitionStr == null) {
-      maxPendingMsgsPerPartitionStr = "1024";
-    }
-    int maxPendingMsgsPerPartition = Integer.parseInt(maxPendingMsgsPerPartitionStr);
-    String enqueueTimeDiffStr = properties.getProperty("eventhub.receiver.filter.timediff");
-    if(enqueueTimeDiffStr == null) {
-      enqueueTimeDiffStr = "0";
-    }
-    int enqueueTimeDiff = Integer.parseInt(enqueueTimeDiffStr);
-    long enqueueTimeFilter = 0;
-    if(enqueueTimeDiff != 0) {
-      enqueueTimeFilter = System.currentTimeMillis() - enqueueTimeDiff*1000;
-    }
-    String consumerGroupName = properties.getProperty("eventhubspout.consumer.group.name");
-    
-    System.out.println("Eventhub spout config: ");
-    System.out.println("  partition count: " + partitionCount);
-    System.out.println("  checkpoint interval: " + checkpointIntervalInSeconds);
-    System.out.println("  receiver credits: " + receiverCredits);
-    spoutConfig = new EventHubSpoutConfig(username, password,
-      namespaceName, entityPath, partitionCount, zkEndpointAddress,
-      checkpointIntervalInSeconds, receiverCredits, maxPendingMsgsPerPartition,
-      enqueueTimeFilter);
-
-    if(targetFqnAddress != null)
-    {
-      spoutConfig.setTargetAddress(targetFqnAddress);      
-    }
-    spoutConfig.setConsumerGroupName(consumerGroupName);
-
-    //set the number of workers to be the same as partition number.
-    //the idea is to have a spout and a partial count bolt co-exist in one
-    //worker to avoid shuffling messages across workers in storm cluster.
-    numWorkers = spoutConfig.getPartitionCount();
-    
-    if(args.length > 0) {
-      //set topology name so that sample Trident topology can use it as stream name.
-      spoutConfig.setTopologyName(args[0]);
-    }
-	}
-  
-  protected EventHubSpout createEventHubSpout() {
-    EventHubSpout eventHubSpout = new EventHubSpout(spoutConfig);
-    return eventHubSpout;
-  }
-	
-  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
-    TopologyBuilder topologyBuilder = new TopologyBuilder();
-
-    topologyBuilder.setSpout("EventHubsSpout", eventHubSpout, spoutConfig.getPartitionCount())
-      .setNumTasks(spoutConfig.getPartitionCount());
-    topologyBuilder.setBolt("PartialCountBolt", new PartialCountBolt(), spoutConfig.getPartitionCount())
-      .localOrShuffleGrouping("EventHubsSpout").setNumTasks(spoutConfig.getPartitionCount());
-    topologyBuilder.setBolt("GlobalCountBolt", new GlobalCountBolt(), 1)
-      .globalGrouping("PartialCountBolt").setNumTasks(1);
-    return topologyBuilder.createTopology();
-  }
-	
-  protected void submitTopology(String[] args, StormTopology topology) throws Exception {
-	  Config config = new Config();
-    config.setDebug(false);
-    //Enable metrics
-    config.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class, 1);
-
-    
-	  if (args != null && args.length > 0) {
-      config.setNumWorkers(numWorkers);
-      StormSubmitter.submitTopology(args[0], config, topology);
-    } else {
-      config.setMaxTaskParallelism(2);
-
-      try (LocalCluster localCluster = new LocalCluster();
-           LocalTopology topo = localCluster.submitTopology("test", config, topology);) {
-        Thread.sleep(5000000);
-      }
-    }
-  }
-  
-  protected void runScenario(String[] args) throws Exception{
-    readEHConfig(args);
-    EventHubSpout eventHubSpout = createEventHubSpout();
-    StormTopology topology = buildTopology(eventHubSpout);
-    submitTopology(args, topology);
-  }
-
-  public static void main(String[] args) throws Exception {
-    EventCount scenario = new EventCount();
-    scenario.runScenario(args);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventHubLoop.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventHubLoop.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventHubLoop.java
deleted file mode 100755
index 665fef9..0000000
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/EventHubLoop.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *******************************************************************************/
-package org.apache.storm.eventhubs.samples;
-
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-
-import org.apache.storm.eventhubs.bolt.EventHubBolt;
-import org.apache.storm.eventhubs.bolt.EventHubBoltConfig;
-import org.apache.storm.eventhubs.spout.EventHubSpout;
-
-/**
- * A sample topology that loops message back to EventHub
- */
-public class EventHubLoop extends EventCount {
-
-  @Override
-  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
-    TopologyBuilder topologyBuilder = new TopologyBuilder();
-
-    topologyBuilder.setSpout("EventHubsSpout", eventHubSpout, spoutConfig.getPartitionCount())
-      .setNumTasks(spoutConfig.getPartitionCount());
-    EventHubBoltConfig boltConfig = new EventHubBoltConfig(spoutConfig.getConnectionString(),
-        spoutConfig.getEntityPath(), true);
-    
-    EventHubBolt eventHubBolt = new EventHubBolt(boltConfig);
-    int boltTasks = spoutConfig.getPartitionCount();
-    topologyBuilder.setBolt("EventHubsBolt", eventHubBolt, boltTasks)
-      .localOrShuffleGrouping("EventHubsSpout").setNumTasks(boltTasks);
-    return topologyBuilder.createTopology();
-  }
-  
-  public static void main(String[] args) throws Exception {
-    EventHubLoop scenario = new EventHubLoop();
-    scenario.runScenario(args);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java
deleted file mode 100755
index e8538c1..0000000
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *******************************************************************************/
-package org.apache.storm.eventhubs.samples;
-
-import org.apache.storm.trident.TridentState;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.operation.builtin.Count;
-import org.apache.storm.trident.operation.builtin.Sum;
-import org.apache.storm.trident.testing.MemoryMapState;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-
-import org.apache.storm.eventhubs.samples.TransactionalTridentEventCount.LoggingFilter;
-import org.apache.storm.eventhubs.spout.EventHubSpout;
-import org.apache.storm.eventhubs.trident.OpaqueTridentEventHubSpout;
-
-/**
- * A simple Trident topology uses OpaqueTridentEventHubSpout
- */
-public class OpaqueTridentEventCount extends EventCount {
-  @Override
-  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
-    TridentTopology topology = new TridentTopology();
-    
-    OpaqueTridentEventHubSpout spout = new OpaqueTridentEventHubSpout(spoutConfig);
-    TridentState state = topology.newStream("stream-" + spoutConfig.getTopologyName(), spout)
-        .parallelismHint(spoutConfig.getPartitionCount())
-        .aggregate(new Count(), new Fields("partial-count"))
-        .persistentAggregate(new MemoryMapState.Factory(), new Fields("partial-count"), new Sum(), new Fields("count"));
-    state.newValuesStream().each(new Fields("count"), new LoggingFilter("got count: ", 10000));
-    return topology.build();
-  }
-  
-  public static void main(String[] args) throws Exception {
-    OpaqueTridentEventCount scenario = new OpaqueTridentEventCount();
-    scenario.runScenario(args);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java
deleted file mode 100755
index 0a5295f..0000000
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *******************************************************************************/
-package org.apache.storm.eventhubs.samples;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-
-import org.apache.storm.eventhubs.spout.EventHubSpout;
-import org.apache.storm.eventhubs.trident.TransactionalTridentEventHubSpout;
-
-import org.apache.storm.trident.TridentState;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.operation.BaseFilter;
-import org.apache.storm.trident.operation.builtin.Count;
-import org.apache.storm.trident.operation.builtin.Sum;
-import org.apache.storm.trident.testing.MemoryMapState;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-/**
- * A simple Trident topology uses TransactionalTridentEventHubSpout
- */
-public class TransactionalTridentEventCount extends EventCount {
-  public static class LoggingFilter extends BaseFilter {
-    private static final long serialVersionUID = 1L;
-    private static final Logger logger = LoggerFactory.getLogger(LoggingFilter.class);
-    private final String prefix;
-    private final long logIntervalMs;
-    private long lastTime;
-    public LoggingFilter(String prefix, int logIntervalMs) {
-      this.prefix = prefix;
-      this.logIntervalMs = logIntervalMs;
-      lastTime = System.nanoTime();
-    }
-
-    @Override
-    public boolean isKeep(TridentTuple tuple) {
-      long now = System.nanoTime();
-      if(logIntervalMs < (now - lastTime) / 1000000) {
-        logger.info(prefix + tuple.toString());
-        lastTime = now;
-      }
-      return false;
-    }
-  }
-  
-  @Override
-  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
-    TridentTopology topology = new TridentTopology();
-    
-    TransactionalTridentEventHubSpout spout = new TransactionalTridentEventHubSpout(spoutConfig);
-    TridentState state = topology.newStream("stream-" + spoutConfig.getTopologyName(), spout)
-        .parallelismHint(spoutConfig.getPartitionCount())
-        .aggregate(new Count(), new Fields("partial-count"))
-        .persistentAggregate(new MemoryMapState.Factory(), new Fields("partial-count"), new Sum(), new Fields("count"));
-    state.newValuesStream().each(new Fields("count"), new LoggingFilter("got count: ", 10000));
-    return topology.build();
-  }
-  
-  public static void main(String[] args) throws Exception {
-    TransactionalTridentEventCount scenario = new TransactionalTridentEventCount();
-    scenario.runScenario(args);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java
deleted file mode 100755
index bc9219e..0000000
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *******************************************************************************/
-package org.apache.storm.eventhubs.samples.bolt;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.storm.utils.TupleUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.storm.Config;
-import org.apache.storm.metric.api.IMetric;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Tuple;
-
-/**
- * Globally count number of messages
- */
-public class GlobalCountBolt extends BaseBasicBolt {
-  private static final long serialVersionUID = 1L;
-  private static final Logger logger = LoggerFactory
-      .getLogger(GlobalCountBolt.class);
-  private long globalCount;
-  private long globalCountDiff;
-  private long lastMetricsTime;
-  private long throughput;
-  
-  @Override
-  public void prepare(Map config, TopologyContext context) {
-    globalCount = 0;
-    globalCountDiff = 0;
-    lastMetricsTime = System.nanoTime();
-    context.registerMetric("GlobalMessageCount", new IMetric() {
-      @Override
-      public Object getValueAndReset() {
-        long now = System.nanoTime();
-        long millis = (now - lastMetricsTime) / 1000000;
-        throughput = globalCountDiff / millis * 1000;
-        Map values = new HashMap();
-        values.put("global_count", globalCount);
-        values.put("throughput", throughput);
-        lastMetricsTime = now;
-        globalCountDiff = 0;
-        return values;
-      }
-  }, (Integer)config.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS));
-  }
-
-  @Override
-  public void execute(Tuple tuple, BasicOutputCollector collector) {
-    if (TupleUtils.isTick(tuple)) {
-      return;
-    }
-
-    int partial = (Integer)tuple.getValueByField("partial_count");
-    globalCount += partial;
-    globalCountDiff += partial;
-    if((globalCountDiff == partial) && (globalCount != globalCountDiff)) {
-      //metrics has just been collected, let's also log it
-      logger.info("Current throughput (messages/second): " + throughput);
-    }
-  }
-
-  @Override
-  public void declareOutputFields(OutputFieldsDeclarer declarer) {
-    
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java
deleted file mode 100755
index eaf2b65..0000000
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *******************************************************************************/
-package org.apache.storm.eventhubs.samples.bolt;
-
-import java.util.Map;
-
-import org.apache.storm.utils.TupleUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-
-/**
- * Partially count number of messages from EventHubs
- */
-public class PartialCountBolt extends BaseBasicBolt {
-  private static final long serialVersionUID = 1L;
-  private static final Logger logger = LoggerFactory
-      .getLogger(PartialCountBolt.class);
-  private static final int PartialCountBatchSize = 1000; 
-  
-  private int partialCount;
-  
-  @Override
-  public void prepare(Map stormConf, TopologyContext context) {
-    partialCount = 0;
-  }
-  
-  @Override
-  public void execute(Tuple tuple, BasicOutputCollector collector) {
-    if (TupleUtils.isTick(tuple)) {
-      return;
-    }
-
-    partialCount++;
-    if(partialCount == PartialCountBatchSize) {
-      collector.emit(new Values(PartialCountBatchSize));
-      partialCount = 0;
-    }
-  }
-
-  @Override
-  public void declareOutputFields(OutputFieldsDeclarer declarer) {
-    declarer.declare(new Fields("partial_count"));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java
new file mode 100755
index 0000000..b0dd33a
--- /dev/null
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/AtMostOnceEventCount.java
@@ -0,0 +1,54 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *******************************************************************************/
+package org.apache.storm.eventhubs.samples;
+
+import java.io.Serializable;
+
+import org.apache.storm.eventhubs.spout.EventHubSpout;
+import org.apache.storm.eventhubs.spout.EventHubSpoutConfig;
+import org.apache.storm.eventhubs.spout.IEventHubReceiver;
+import org.apache.storm.eventhubs.spout.IPartitionManager;
+import org.apache.storm.eventhubs.spout.IPartitionManagerFactory;
+import org.apache.storm.eventhubs.spout.IStateStore;
+import org.apache.storm.eventhubs.spout.SimplePartitionManager;
+
+public class AtMostOnceEventCount extends EventCount implements Serializable {
+  @Override
+  protected EventHubSpout createEventHubSpout() {
+    IPartitionManagerFactory pmFactory = new IPartitionManagerFactory() {
+      private static final long serialVersionUID = 1L;
+
+      @Override
+      public IPartitionManager create(EventHubSpoutConfig spoutConfig,
+          String partitionId, IStateStore stateStore,
+          IEventHubReceiver receiver) {
+        return new SimplePartitionManager(spoutConfig, partitionId,
+            stateStore, receiver);
+      }
+    };
+    EventHubSpout eventHubSpout = new EventHubSpout(
+        spoutConfig, null, pmFactory, null);
+    return eventHubSpout;
+  }
+  
+  public static void main(String[] args) throws Exception {
+    AtMostOnceEventCount scenario = new AtMostOnceEventCount();
+
+    scenario.runScenario(args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java
new file mode 100755
index 0000000..ae15634
--- /dev/null
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java
@@ -0,0 +1,157 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *******************************************************************************/
+package org.apache.storm.eventhubs.samples;
+
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+
+import org.apache.storm.eventhubs.samples.bolt.GlobalCountBolt;
+import org.apache.storm.eventhubs.samples.bolt.PartialCountBolt;
+import org.apache.storm.eventhubs.spout.EventHubSpout;
+import org.apache.storm.eventhubs.spout.EventHubSpoutConfig;
+
+import java.io.FileReader;
+import java.util.Properties;
+
+/**
+ * The basic scenario topology that uses EventHubSpout with PartialCountBolt
+ * and GlobalCountBolt.
+ * To submit this topology:
+ * storm jar {jarfile} {classname} {topologyname} {spoutconffile}
+ */
+public class EventCount {
+  protected EventHubSpoutConfig spoutConfig;
+  protected int numWorkers;
+  
+  public EventCount() {
+  }
+  
+  protected void readEHConfig(String[] args) throws Exception {
+	  Properties properties = new Properties();
+    if(args.length > 1) {
+      properties.load(new FileReader(args[1]));
+    }
+    else {
+      properties.load(EventCount.class.getClassLoader().getResourceAsStream(
+          "Config.properties"));
+    }
+
+    String username = properties.getProperty("eventhubspout.username");
+    String password = properties.getProperty("eventhubspout.password");
+    String namespaceName = properties.getProperty("eventhubspout.namespace");
+    String entityPath = properties.getProperty("eventhubspout.entitypath");
+    String targetFqnAddress = properties.getProperty("eventhubspout.targetfqnaddress");
+    String zkEndpointAddress = properties.getProperty("zookeeper.connectionstring");
+    int partitionCount = Integer.parseInt(properties.getProperty("eventhubspout.partitions.count"));
+    int checkpointIntervalInSeconds = Integer.parseInt(properties.getProperty("eventhubspout.checkpoint.interval"));
+    int receiverCredits = Integer.parseInt(properties.getProperty("eventhub.receiver.credits"));
+    String maxPendingMsgsPerPartitionStr = properties.getProperty("eventhubspout.max.pending.messages.per.partition");
+    if(maxPendingMsgsPerPartitionStr == null) {
+      maxPendingMsgsPerPartitionStr = "1024";
+    }
+    int maxPendingMsgsPerPartition = Integer.parseInt(maxPendingMsgsPerPartitionStr);
+    String enqueueTimeDiffStr = properties.getProperty("eventhub.receiver.filter.timediff");
+    if(enqueueTimeDiffStr == null) {
+      enqueueTimeDiffStr = "0";
+    }
+    int enqueueTimeDiff = Integer.parseInt(enqueueTimeDiffStr);
+    long enqueueTimeFilter = 0;
+    if(enqueueTimeDiff != 0) {
+      enqueueTimeFilter = System.currentTimeMillis() - enqueueTimeDiff*1000;
+    }
+    String consumerGroupName = properties.getProperty("eventhubspout.consumer.group.name");
+    
+    System.out.println("Eventhub spout config: ");
+    System.out.println("  partition count: " + partitionCount);
+    System.out.println("  checkpoint interval: " + checkpointIntervalInSeconds);
+    System.out.println("  receiver credits: " + receiverCredits);
+    spoutConfig = new EventHubSpoutConfig(username, password,
+      namespaceName, entityPath, partitionCount, zkEndpointAddress,
+      checkpointIntervalInSeconds, receiverCredits, maxPendingMsgsPerPartition,
+      enqueueTimeFilter);
+
+    if(targetFqnAddress != null)
+    {
+      spoutConfig.setTargetAddress(targetFqnAddress);      
+    }
+    spoutConfig.setConsumerGroupName(consumerGroupName);
+
+    //set the number of workers to be the same as partition number.
+    //the idea is to have a spout and a partial count bolt co-exist in one
+    //worker to avoid shuffling messages across workers in storm cluster.
+    numWorkers = spoutConfig.getPartitionCount();
+    
+    if(args.length > 0) {
+      //set topology name so that sample Trident topology can use it as stream name.
+      spoutConfig.setTopologyName(args[0]);
+    }
+	}
+  
+  protected EventHubSpout createEventHubSpout() {
+    EventHubSpout eventHubSpout = new EventHubSpout(spoutConfig);
+    return eventHubSpout;
+  }
+	
+  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
+    TopologyBuilder topologyBuilder = new TopologyBuilder();
+
+    topologyBuilder.setSpout("EventHubsSpout", eventHubSpout, spoutConfig.getPartitionCount())
+      .setNumTasks(spoutConfig.getPartitionCount());
+    topologyBuilder.setBolt("PartialCountBolt", new PartialCountBolt(), spoutConfig.getPartitionCount())
+      .localOrShuffleGrouping("EventHubsSpout").setNumTasks(spoutConfig.getPartitionCount());
+    topologyBuilder.setBolt("GlobalCountBolt", new GlobalCountBolt(), 1)
+      .globalGrouping("PartialCountBolt").setNumTasks(1);
+    return topologyBuilder.createTopology();
+  }
+	
+  protected void submitTopology(String[] args, StormTopology topology) throws Exception {
+	  Config config = new Config();
+    config.setDebug(false);
+    //Enable metrics
+    config.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class, 1);
+
+    
+	  if (args != null && args.length > 0) {
+      config.setNumWorkers(numWorkers);
+      StormSubmitter.submitTopology(args[0], config, topology);
+    } else {
+      config.setMaxTaskParallelism(2);
+
+      try (LocalCluster localCluster = new LocalCluster();
+           LocalTopology topo = localCluster.submitTopology("test", config, topology);) {
+        Thread.sleep(5000000);
+      }
+    }
+  }
+  
+  protected void runScenario(String[] args) throws Exception{
+    readEHConfig(args);
+    EventHubSpout eventHubSpout = createEventHubSpout();
+    StormTopology topology = buildTopology(eventHubSpout);
+    submitTopology(args, topology);
+  }
+
+  public static void main(String[] args) throws Exception {
+    EventCount scenario = new EventCount();
+    scenario.runScenario(args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventHubLoop.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventHubLoop.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventHubLoop.java
new file mode 100755
index 0000000..665fef9
--- /dev/null
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventHubLoop.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *******************************************************************************/
+package org.apache.storm.eventhubs.samples;
+
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+
+import org.apache.storm.eventhubs.bolt.EventHubBolt;
+import org.apache.storm.eventhubs.bolt.EventHubBoltConfig;
+import org.apache.storm.eventhubs.spout.EventHubSpout;
+
+/**
+ * A sample topology that loops message back to EventHub
+ */
+public class EventHubLoop extends EventCount {
+
+  @Override
+  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
+    TopologyBuilder topologyBuilder = new TopologyBuilder();
+
+    topologyBuilder.setSpout("EventHubsSpout", eventHubSpout, spoutConfig.getPartitionCount())
+      .setNumTasks(spoutConfig.getPartitionCount());
+    EventHubBoltConfig boltConfig = new EventHubBoltConfig(spoutConfig.getConnectionString(),
+        spoutConfig.getEntityPath(), true);
+    
+    EventHubBolt eventHubBolt = new EventHubBolt(boltConfig);
+    int boltTasks = spoutConfig.getPartitionCount();
+    topologyBuilder.setBolt("EventHubsBolt", eventHubBolt, boltTasks)
+      .localOrShuffleGrouping("EventHubsSpout").setNumTasks(boltTasks);
+    return topologyBuilder.createTopology();
+  }
+  
+  public static void main(String[] args) throws Exception {
+    EventHubLoop scenario = new EventHubLoop();
+    scenario.runScenario(args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java
new file mode 100755
index 0000000..e8538c1
--- /dev/null
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/OpaqueTridentEventCount.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *******************************************************************************/
+package org.apache.storm.eventhubs.samples;
+
+import org.apache.storm.trident.TridentState;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.operation.builtin.Count;
+import org.apache.storm.trident.operation.builtin.Sum;
+import org.apache.storm.trident.testing.MemoryMapState;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.tuple.Fields;
+
+import org.apache.storm.eventhubs.samples.TransactionalTridentEventCount.LoggingFilter;
+import org.apache.storm.eventhubs.spout.EventHubSpout;
+import org.apache.storm.eventhubs.trident.OpaqueTridentEventHubSpout;
+
+/**
+ * A simple Trident topology uses OpaqueTridentEventHubSpout
+ */
+public class OpaqueTridentEventCount extends EventCount {
+  @Override
+  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
+    TridentTopology topology = new TridentTopology();
+    
+    OpaqueTridentEventHubSpout spout = new OpaqueTridentEventHubSpout(spoutConfig);
+    TridentState state = topology.newStream("stream-" + spoutConfig.getTopologyName(), spout)
+        .parallelismHint(spoutConfig.getPartitionCount())
+        .aggregate(new Count(), new Fields("partial-count"))
+        .persistentAggregate(new MemoryMapState.Factory(), new Fields("partial-count"), new Sum(), new Fields("count"));
+    state.newValuesStream().each(new Fields("count"), new LoggingFilter("got count: ", 10000));
+    return topology.build();
+  }
+  
+  public static void main(String[] args) throws Exception {
+    OpaqueTridentEventCount scenario = new OpaqueTridentEventCount();
+    scenario.runScenario(args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java
new file mode 100755
index 0000000..0a5295f
--- /dev/null
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/TransactionalTridentEventCount.java
@@ -0,0 +1,81 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *******************************************************************************/
+package org.apache.storm.eventhubs.samples;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.tuple.Fields;
+
+import org.apache.storm.eventhubs.spout.EventHubSpout;
+import org.apache.storm.eventhubs.trident.TransactionalTridentEventHubSpout;
+
+import org.apache.storm.trident.TridentState;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.operation.BaseFilter;
+import org.apache.storm.trident.operation.builtin.Count;
+import org.apache.storm.trident.operation.builtin.Sum;
+import org.apache.storm.trident.testing.MemoryMapState;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+/**
+ * A simple Trident topology uses TransactionalTridentEventHubSpout
+ */
+public class TransactionalTridentEventCount extends EventCount {
+  public static class LoggingFilter extends BaseFilter {
+    private static final long serialVersionUID = 1L;
+    private static final Logger logger = LoggerFactory.getLogger(LoggingFilter.class);
+    private final String prefix;
+    private final long logIntervalMs;
+    private long lastTime;
+    public LoggingFilter(String prefix, int logIntervalMs) {
+      this.prefix = prefix;
+      this.logIntervalMs = logIntervalMs;
+      lastTime = System.nanoTime();
+    }
+
+    @Override
+    public boolean isKeep(TridentTuple tuple) {
+      long now = System.nanoTime();
+      if(logIntervalMs < (now - lastTime) / 1000000) {
+        logger.info(prefix + tuple.toString());
+        lastTime = now;
+      }
+      return false;
+    }
+  }
+  
+  @Override
+  protected StormTopology buildTopology(EventHubSpout eventHubSpout) {
+    TridentTopology topology = new TridentTopology();
+    
+    TransactionalTridentEventHubSpout spout = new TransactionalTridentEventHubSpout(spoutConfig);
+    TridentState state = topology.newStream("stream-" + spoutConfig.getTopologyName(), spout)
+        .parallelismHint(spoutConfig.getPartitionCount())
+        .aggregate(new Count(), new Fields("partial-count"))
+        .persistentAggregate(new MemoryMapState.Factory(), new Fields("partial-count"), new Sum(), new Fields("count"));
+    state.newValuesStream().each(new Fields("count"), new LoggingFilter("got count: ", 10000));
+    return topology.build();
+  }
+  
+  public static void main(String[] args) throws Exception {
+    TransactionalTridentEventCount scenario = new TransactionalTridentEventCount();
+    scenario.runScenario(args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java
new file mode 100755
index 0000000..bc9219e
--- /dev/null
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/GlobalCountBolt.java
@@ -0,0 +1,88 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *******************************************************************************/
+package org.apache.storm.eventhubs.samples.bolt;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.storm.utils.TupleUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.storm.Config;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Tuple;
+
+/**
+ * Globally count number of messages
+ */
+public class GlobalCountBolt extends BaseBasicBolt {
+  private static final long serialVersionUID = 1L;
+  private static final Logger logger = LoggerFactory
+      .getLogger(GlobalCountBolt.class);
+  private long globalCount;
+  private long globalCountDiff;
+  private long lastMetricsTime;
+  private long throughput;
+  
+  @Override
+  public void prepare(Map config, TopologyContext context) {
+    globalCount = 0;
+    globalCountDiff = 0;
+    lastMetricsTime = System.nanoTime();
+    context.registerMetric("GlobalMessageCount", new IMetric() {
+      @Override
+      public Object getValueAndReset() {
+        long now = System.nanoTime();
+        long millis = (now - lastMetricsTime) / 1000000;
+        throughput = globalCountDiff / millis * 1000;
+        Map values = new HashMap();
+        values.put("global_count", globalCount);
+        values.put("throughput", throughput);
+        lastMetricsTime = now;
+        globalCountDiff = 0;
+        return values;
+      }
+  }, (Integer)config.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS));
+  }
+
+  @Override
+  public void execute(Tuple tuple, BasicOutputCollector collector) {
+    if (TupleUtils.isTick(tuple)) {
+      return;
+    }
+
+    int partial = (Integer)tuple.getValueByField("partial_count");
+    globalCount += partial;
+    globalCountDiff += partial;
+    if((globalCountDiff == partial) && (globalCount != globalCountDiff)) {
+      //metrics has just been collected, let's also log it
+      logger.info("Current throughput (messages/second): " + throughput);
+    }
+  }
+
+  @Override
+  public void declareOutputFields(OutputFieldsDeclarer declarer) {
+    
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java
new file mode 100755
index 0000000..eaf2b65
--- /dev/null
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/bolt/PartialCountBolt.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *******************************************************************************/
+package org.apache.storm.eventhubs.samples.bolt;
+
+import java.util.Map;
+
+import org.apache.storm.utils.TupleUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+
+/**
+ * Partially count number of messages from EventHubs
+ */
+public class PartialCountBolt extends BaseBasicBolt {
+  private static final long serialVersionUID = 1L;
+  private static final Logger logger = LoggerFactory
+      .getLogger(PartialCountBolt.class);
+  private static final int PartialCountBatchSize = 1000; 
+  
+  private int partialCount;
+  
+  @Override
+  public void prepare(Map stormConf, TopologyContext context) {
+    partialCount = 0;
+  }
+  
+  @Override
+  public void execute(Tuple tuple, BasicOutputCollector collector) {
+    if (TupleUtils.isTick(tuple)) {
+      return;
+    }
+
+    partialCount++;
+    if(partialCount == PartialCountBatchSize) {
+      collector.emit(new Values(PartialCountBatchSize));
+      partialCount = 0;
+    }
+  }
+
+  @Override
+  public void declareOutputFields(OutputFieldsDeclarer declarer) {
+    declarer.declare(new Fields("partial_count"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml
index 3d6aa0c..fd6bfe5 100644
--- a/external/storm-hbase/pom.xml
+++ b/external/storm-hbase/pom.xml
@@ -44,7 +44,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml
index 281732e..8b3a792 100644
--- a/external/storm-hdfs/pom.xml
+++ b/external/storm-hdfs/pom.xml
@@ -45,7 +45,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
             <exclusions>
@@ -57,9 +57,10 @@
                 </exclusion>
             </exclusions>
         </dependency>
+
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <type>test-jar</type>
             <scope>test</scope>
@@ -72,6 +73,7 @@
                 </exclusion>
             </exclusions>
         </dependency>
+
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java
index a4c88ce..9acae35 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java
@@ -19,7 +19,7 @@ package org.apache.storm.hdfs.blobstore;
 
 import org.apache.storm.Config;
 import org.apache.storm.blobstore.BlobStoreFile;
-import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -139,7 +139,7 @@ public class HdfsBlobStoreImpl {
         }
 
         Object shouldCleanup = conf.get(Config.BLOBSTORE_CLEANUP_ENABLE);
-        if (Utils.getBoolean(shouldCleanup, false)) {
+        if (ObjectReader.getBoolean(shouldCleanup, false)) {
             LOG.debug("Starting hdfs blobstore cleaner");
             _cleanup = new TimerTask() {
                 @Override


[02/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java
new file mode 100644
index 0000000..d7b20d6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.io.IOException;
+import java.util.Map;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KerberosSaslClientHandler extends SimpleChannelUpstreamHandler {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslClientHandler.class);
+    private ISaslClient client;
+    long start_time;
+    /** Used for client or server's token to send or receive from each other. */
+    private Map storm_conf;
+    private String jaas_section;
+    private String host;
+
+    public KerberosSaslClientHandler(ISaslClient client, Map storm_conf, String jaas_section, String host) throws IOException {
+        this.client = client;
+        this.storm_conf = storm_conf;
+        this.jaas_section = jaas_section;
+        this.host = host;
+        start_time = System.currentTimeMillis();
+    }
+
+    @Override
+    public void channelConnected(ChannelHandlerContext ctx,
+                                 ChannelStateEvent event) {
+        // register the newly established channel
+        Channel channel = ctx.getChannel();
+        client.channelConnected(channel);
+
+        LOG.info("Connection established from {} to {}",
+                 channel.getLocalAddress(), channel.getRemoteAddress());
+
+        try {
+            KerberosSaslNettyClient saslNettyClient = KerberosSaslNettyClientState.getKerberosSaslNettyClient
+                .get(channel);
+
+            if (saslNettyClient == null) {
+                LOG.debug("Creating saslNettyClient now for channel: {}",
+                          channel);
+                saslNettyClient = new KerberosSaslNettyClient(storm_conf, jaas_section, host);
+                KerberosSaslNettyClientState.getKerberosSaslNettyClient.set(channel,
+                                                                            saslNettyClient);
+            }
+            LOG.debug("Going to initiate Kerberos negotiations.");
+            byte[] initialChallenge = saslNettyClient.saslResponse(new SaslMessageToken(new byte[0]));
+            LOG.debug("Sending initial challenge: {}", initialChallenge);
+            channel.write(new SaslMessageToken(initialChallenge));
+        } catch (Exception e) {
+            LOG.error("Failed to authenticate with server due to error: ",
+                      e);
+        }
+        return;
+
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent event)
+        throws Exception {
+        LOG.debug("send/recv time (ms): {}",
+                  (System.currentTimeMillis() - start_time));
+
+        Channel channel = ctx.getChannel();
+
+        // Generate SASL response to server using Channel-local SASL client.
+        KerberosSaslNettyClient saslNettyClient = KerberosSaslNettyClientState.getKerberosSaslNettyClient
+            .get(channel);
+        if (saslNettyClient == null) {
+            throw new Exception("saslNettyClient was unexpectedly null for channel:" + channel);
+        }
+
+        // examine the response message from server
+        if (event.getMessage() instanceof ControlMessage) {
+            ControlMessage msg = (ControlMessage) event.getMessage();
+            if (msg == ControlMessage.SASL_COMPLETE_REQUEST) {
+                LOG.debug("Server has sent us the SaslComplete message. Allowing normal work to proceed.");
+
+                if (!saslNettyClient.isComplete()) {
+                    String message = "Server returned a Sasl-complete message, but as far as we can tell, we are not authenticated yet.";
+                    LOG.error(message);
+                    throw new Exception(message);
+                }
+                ctx.getPipeline().remove(this);
+                this.client.channelReady();
+
+                // We call fireMessageReceived since the client is allowed to
+                // perform this request. The client's request will now proceed
+                // to the next pipeline component namely StormClientHandler.
+                Channels.fireMessageReceived(ctx, msg);
+            } else {
+                LOG.warn("Unexpected control message: {}", msg);
+            }
+            return;
+        }
+        else if (event.getMessage() instanceof SaslMessageToken) {
+            SaslMessageToken saslTokenMessage = (SaslMessageToken) event
+                .getMessage();
+            LOG.debug("Responding to server's token of length: {}",
+                      saslTokenMessage.getSaslToken().length);
+
+            // Generate SASL response (but we only actually send the response if
+            // it's non-null.
+            byte[] responseToServer = saslNettyClient
+                .saslResponse(saslTokenMessage);
+            if (responseToServer == null) {
+                // If we generate a null response, then authentication has completed
+                // (if not, warn), and return without sending a response back to the
+                // server.
+                LOG.debug("Response to server is null: authentication should now be complete.");
+                if (!saslNettyClient.isComplete()) {
+                    LOG.warn("Generated a null response, but authentication is not complete.");
+                    throw new Exception("Our reponse to the server is null, but as far as we can tell, we are not authenticated yet.");
+                }
+                this.client.channelReady();
+                return;
+            } else {
+                LOG.debug("Response to server token has length: {}",
+                          responseToServer.length);
+            }
+            // Construct a message containing the SASL response and send it to the
+            // server.
+            SaslMessageToken saslResponse = new SaslMessageToken(responseToServer);
+            channel.write(saslResponse);
+        } else {
+            LOG.error("Unexpected message from server: {}", event.getMessage());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java
new file mode 100644
index 0000000..2efcb19
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.apache.storm.Config;
+import org.apache.storm.security.auth.AuthUtils;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.TreeMap;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import org.apache.zookeeper.server.auth.KerberosName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements SASL logic for storm worker client processes.
+ */
+public class KerberosSaslNettyClient {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslNettyClient.class);
+
+    /**
+     * Used to respond to server's counterpart, SaslServer with SASL tokens
+     * represented as byte arrays.
+     */
+    private SaslClient saslClient;
+    private Subject subject;
+    private String jaas_section;
+
+    /**
+     * Create a KerberosSaslNettyClient for authentication with servers.
+     */
+    public KerberosSaslNettyClient(Map storm_conf, String jaas_section, String host) {
+        LOG.debug("KerberosSaslNettyClient: Creating SASL {} client to authenticate to server ",
+                  SaslUtils.KERBEROS);
+
+        LOG.info("Creating Kerberos Client.");
+
+        Configuration login_conf;
+        try {
+            login_conf = AuthUtils.GetConfiguration(storm_conf);
+        }
+        catch (Throwable t) {
+            LOG.error("Failed to get login_conf: ", t);
+            throw t;
+        }
+        LOG.debug("KerberosSaslNettyClient: authmethod {}", SaslUtils.KERBEROS);
+
+        SaslClientCallbackHandler ch = new SaslClientCallbackHandler();
+
+        subject = null;
+        try {
+            LOG.debug("Setting Configuration to login_config: {}", login_conf);
+            //specify a configuration object to be used
+            Configuration.setConfiguration(login_conf);
+            //now login
+            LOG.debug("Trying to login.");
+            Login login = new Login(jaas_section, ch);
+            subject = login.getSubject();
+            LOG.debug("Got Subject: {}", subject.toString());
+        } catch (LoginException ex) {
+            LOG.error("Client failed to login in principal:" + ex, ex);
+            throw new RuntimeException(ex);
+        }
+
+        //check the credential of our principal
+        if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) {
+            LOG.error("Failed to verify user principal.");
+            throw new RuntimeException("Fail to verify user principal with section \"" +
+                                       jaas_section +
+                                       "\" in login configuration file " +
+                                       login_conf);
+        }
+
+        String serviceName = null;
+        try {
+            serviceName = AuthUtils.get(login_conf, jaas_section, "serviceName");
+        }
+        catch (IOException e) {
+            LOG.error("Failed to get service name.", e);
+            throw new RuntimeException(e);
+        }
+
+        try {
+            Principal principal = (Principal)subject.getPrincipals().toArray()[0];
+            final String fPrincipalName = principal.getName();
+            final String fHost = host;
+            final String fServiceName = serviceName;
+            final CallbackHandler fch = ch;
+            LOG.debug("Kerberos Client with principal: {}, host: {}", fPrincipalName, fHost);
+            saslClient = Subject.doAs(subject, new PrivilegedExceptionAction<SaslClient>() {
+                    public SaslClient run() {
+                        try {
+                            Map<String, String> props = new TreeMap<String,String>();
+                            props.put(Sasl.QOP, "auth");
+                            props.put(Sasl.SERVER_AUTH, "false");
+                            return Sasl.createSaslClient(
+                                new String[] { SaslUtils.KERBEROS },
+                                fPrincipalName,
+                                fServiceName,
+                                fHost,
+                                props, fch);
+                        }
+                        catch (Exception e) {
+                            LOG.error("Subject failed to create sasl client.", e);
+                            return null;
+                        }
+                    }
+                });
+            LOG.info("Got Client: {}", saslClient);
+
+        } catch (PrivilegedActionException e) {
+            LOG.error("KerberosSaslNettyClient: Could not create Sasl Netty Client.");
+            throw new RuntimeException(e);
+        }
+    }
+
+    public boolean isComplete() {
+        return saslClient.isComplete();
+    }
+
+    /**
+     * Respond to server's SASL token.
+     *
+     * @param saslTokenMessage
+     *            contains server's SASL token
+     * @return client's response SASL token
+     */
+    public byte[] saslResponse(SaslMessageToken saslTokenMessage) {
+        try {
+            final SaslMessageToken fSaslTokenMessage = saslTokenMessage;
+            byte [] retval = Subject.doAs(subject, new PrivilegedExceptionAction<byte[]>() {
+                    public byte[] run() {
+                        try {
+                            byte[] retval = saslClient.evaluateChallenge(fSaslTokenMessage
+                                                                         .getSaslToken());
+                            return retval;
+                        } catch (SaslException e) {
+                            LOG.error("saslResponse: Failed to respond to SASL server's token:",
+                                      e);
+                            throw new RuntimeException(e);
+                        }
+                    }
+                });
+            return retval;
+        }
+        catch (PrivilegedActionException e) {
+            LOG.error("Failed to generate response for token: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Implementation of javax.security.auth.callback.CallbackHandler that works
+     * with Storm topology tokens.
+     */
+    private static class SaslClientCallbackHandler implements CallbackHandler {
+
+        /**
+         * Implementation used to respond to SASL tokens from server.
+         *
+         * @param callbacks
+         *            objects that indicate what credential information the
+         *            server's SaslServer requires from the client.
+         * @throws UnsupportedCallbackException
+         */
+        public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                LOG.info("Kerberos Client Callback Handler got callback: {}", callback.getClass());
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java
new file mode 100644
index 0000000..dc76b0d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelLocal;
+
+final class KerberosSaslNettyClientState {
+
+    public static final ChannelLocal<KerberosSaslNettyClient> getKerberosSaslNettyClient = new ChannelLocal<KerberosSaslNettyClient>() {
+        protected KerberosSaslNettyClient initialValue(Channel channel) {
+            return null;
+        }
+    };
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java
new file mode 100644
index 0000000..72486ef
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.KerberosPrincipalToLocal;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import org.apache.zookeeper.server.auth.KerberosName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+class KerberosSaslNettyServer {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslNettyServer.class);
+
+    private SaslServer saslServer;
+    private Subject subject;
+    private List<String> authorizedUsers;
+
+    KerberosSaslNettyServer(Map storm_conf, String jaas_section, List<String> authorizedUsers) {
+        this.authorizedUsers = authorizedUsers;
+        LOG.debug("Getting Configuration.");
+        Configuration login_conf;
+        try {
+            login_conf = AuthUtils.GetConfiguration(storm_conf);
+        }
+        catch (Throwable t) {
+            LOG.error("Failed to get login_conf: ", t);
+            throw t;
+        }
+
+        LOG.debug("KerberosSaslNettyServer: authmethod {}", SaslUtils.KERBEROS);
+
+        KerberosSaslCallbackHandler ch = new KerberosSaslNettyServer.KerberosSaslCallbackHandler(authorizedUsers);
+
+        //login our principal
+        subject = null;
+        try {
+            LOG.debug("Setting Configuration to login_config: {}", login_conf);
+            //specify a configuration object to be used
+            Configuration.setConfiguration(login_conf);
+            //now login
+            LOG.debug("Trying to login.");
+            Login login = new Login(jaas_section, ch);
+            subject = login.getSubject();
+            LOG.debug("Got Subject: {}", subject.toString());
+        } catch (LoginException ex) {
+            LOG.error("Server failed to login in principal:", ex);
+            throw new RuntimeException(ex);
+        }
+
+        //check the credential of our principal
+        if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) {
+            LOG.error("Failed to verifyuser principal.");
+            throw new RuntimeException("Fail to verify user principal with section \""
+                                       + jaas_section
+                                       + "\" in login configuration file "
+                                       + login_conf);
+        }
+
+        try {
+            LOG.info("Creating Kerberos Server.");
+            final CallbackHandler fch = ch;
+            Principal p = (Principal)subject.getPrincipals().toArray()[0];
+            KerberosName kName = new KerberosName(p.getName());
+            final String fHost = kName.getHostName();
+            final String fServiceName = kName.getServiceName();
+            LOG.debug("Server with host: {}", fHost);
+            saslServer =
+                Subject.doAs(subject, new PrivilegedExceptionAction<SaslServer>() {
+                        public SaslServer run() {
+                            try {
+                                Map<String, String> props = new TreeMap<String,String>();
+                                props.put(Sasl.QOP, "auth");
+                                props.put(Sasl.SERVER_AUTH, "false");
+                                return Sasl.createSaslServer(SaslUtils.KERBEROS,
+                                                             fServiceName,
+                                                             fHost, props, fch);
+                            }
+                            catch (Exception e) {
+                                LOG.error("Subject failed to create sasl server.", e);
+                                return null;
+                            }
+                        }
+                    });
+            LOG.info("Got Server: {}", saslServer);
+
+        } catch (PrivilegedActionException e) {
+            LOG.error("KerberosSaslNettyServer: Could not create SaslServer: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public boolean isComplete() {
+        return saslServer.isComplete();
+    }
+
+    public String getUserName() {
+        return saslServer.getAuthorizationID();
+    }
+
+    /** CallbackHandler for SASL DIGEST-MD5 mechanism */
+    public static class KerberosSaslCallbackHandler implements CallbackHandler {
+
+        /** Used to authenticate the clients */
+        private List<String> authorizedUsers;
+
+        public KerberosSaslCallbackHandler(List<String> authorizedUsers) {
+            LOG.debug("KerberosSaslCallback: Creating KerberosSaslCallback handler.");
+            this.authorizedUsers = authorizedUsers;
+        }
+
+        @Override
+        public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                LOG.info("Kerberos Callback Handler got callback: {}", callback.getClass());
+                if(callback instanceof AuthorizeCallback) {
+                    AuthorizeCallback ac = (AuthorizeCallback)callback;
+                    if(!ac.getAuthenticationID().equals(ac.getAuthorizationID())) {
+                        LOG.debug("{} != {}", ac.getAuthenticationID(), ac.getAuthorizationID());
+                        continue;
+                    }
+
+                    LOG.debug("Authorized Users: {}", authorizedUsers);
+                    LOG.debug("Checking authorization for: {}", ac.getAuthorizationID());
+                    for(String user : authorizedUsers) {
+                        String requester = ac.getAuthorizationID();
+
+                        KerberosPrincipal principal = new KerberosPrincipal(requester);
+                        requester = new KerberosPrincipalToLocal().toLocal(principal);
+
+                        if(requester.equals(user) ) {
+                            ac.setAuthorized(true);
+                            break;
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Used by SaslTokenMessage::processToken() to respond to server SASL
+     * tokens.
+     *
+     * @param token
+     *            Server's SASL token
+     * @return token to send back to the server.
+     */
+    public byte[] response(final byte[] token) {
+        try {
+            byte [] retval = Subject.doAs(subject, new PrivilegedExceptionAction<byte[]>() {
+                    public byte[] run(){
+                        try {
+                            LOG.debug("response: Responding to input token of length: {}",
+                                      token.length);
+                            byte[] retval = saslServer.evaluateResponse(token);
+                            return retval;
+                        } catch (SaslException e) {
+                            LOG.error("response: Failed to evaluate client token of length: {} : {}",
+                                      token.length, e);
+                            throw new RuntimeException(e);
+                        }
+                    }
+                });
+            return retval;
+        }
+        catch (PrivilegedActionException e) {
+            LOG.error("Failed to generate response for token: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java
new file mode 100644
index 0000000..2ee2bf4
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelLocal;
+
+final class KerberosSaslNettyServerState {
+
+    public static final ChannelLocal<KerberosSaslNettyServer> getKerberosSaslNettyServer = new ChannelLocal<KerberosSaslNettyServer>() {
+            protected KerberosSaslNettyServer initialValue(Channel channel) {
+                return null;
+            }
+        };
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java
new file mode 100644
index 0000000..14ac172
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KerberosSaslServerHandler extends SimpleChannelUpstreamHandler {
+
+    ISaslServer server;
+    /** Used for client or server's token to send or receive from each other. */
+    private Map storm_conf;
+    private String jaas_section;
+    private List<String> authorizedUsers;
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslServerHandler.class);
+
+    public KerberosSaslServerHandler(ISaslServer server, Map storm_conf, String jaas_section, List<String> authorizedUsers) throws IOException {
+        this.server = server;
+        this.storm_conf = storm_conf;
+        this.jaas_section = jaas_section;
+        this.authorizedUsers = authorizedUsers;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+        throws Exception {
+        Object msg = e.getMessage();
+        if (msg == null) {
+            return;
+        }
+
+        Channel channel = ctx.getChannel();
+
+
+        if (msg instanceof SaslMessageToken) {
+            // initialize server-side SASL functionality, if we haven't yet
+            // (in which case we are looking at the first SASL message from the
+            // client).
+            try {
+                LOG.debug("Got SaslMessageToken!");
+
+                KerberosSaslNettyServer saslNettyServer = KerberosSaslNettyServerState.getKerberosSaslNettyServer
+                    .get(channel);
+                if (saslNettyServer == null) {
+                    LOG.debug("No saslNettyServer for {}  yet; creating now, with topology token: ", channel);
+                    try {
+                        saslNettyServer = new KerberosSaslNettyServer(storm_conf, jaas_section, authorizedUsers);
+                        KerberosSaslNettyServerState.getKerberosSaslNettyServer.set(channel,
+                                                                                    saslNettyServer);
+                    } catch (RuntimeException ioe) {
+                        LOG.error("Error occurred while creating saslNettyServer on server {} for client {}",
+                                  channel.getLocalAddress(), channel.getRemoteAddress());
+                        throw ioe;
+                    }
+                } else {
+                    LOG.debug("Found existing saslNettyServer on server: {} for client {}",
+                              channel.getLocalAddress(), channel.getRemoteAddress());
+                }
+
+                byte[] responseBytes = saslNettyServer.response(((SaslMessageToken) msg)
+                                                                .getSaslToken());
+
+                SaslMessageToken saslTokenMessageRequest = new SaslMessageToken(responseBytes);
+
+                if(saslTokenMessageRequest.getSaslToken() == null) {
+                    channel.write(ControlMessage.SASL_COMPLETE_REQUEST);
+                } else {
+                    // Send response to client.
+                    channel.write(saslTokenMessageRequest);
+                }
+
+                if (saslNettyServer.isComplete()) {
+                    // If authentication of client is complete, we will also send a
+                    // SASL-Complete message to the client.
+                    LOG.info("SASL authentication is complete for client with username: {}",
+                             saslNettyServer.getUserName());
+                    channel.write(ControlMessage.SASL_COMPLETE_REQUEST);
+                    LOG.debug("Removing SaslServerHandler from pipeline since SASL authentication is complete.");
+                    ctx.getPipeline().remove(this);
+                    server.authenticated(channel);
+                }
+                return;
+            }
+            catch (Exception ex) {
+                LOG.error("Failed to handle SaslMessageToken: ", ex);
+                throw ex;
+            }
+        } else {
+            // Client should not be sending other-than-SASL messages before
+            // SaslServerHandler has removed itself from the pipeline. Such
+            // non-SASL requests will be denied by the Authorize channel handler
+            // (the next handler upstream in the server pipeline) if SASL
+            // authentication has not completed.
+            LOG.warn("Sending upstream an unexpected non-SASL message : {}",
+                     msg);
+            Channels.fireMessageReceived(ctx, msg);
+        }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
+        if(server != null) {
+            server.closeChannel(e.getChannel());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java
new file mode 100644
index 0000000..718c8f3
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java
@@ -0,0 +1,411 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.messaging.netty;
+
+/**
+ * This class is responsible for refreshing Kerberos credentials for
+ * logins for both Zookeeper client and server.
+ * See ZooKeeperSaslServer for server-side usage.
+ * See ZooKeeperSaslClient for client-side usage.
+ * This class is a copied from https://github.com/apache/zookeeper/blob/branch-3.4/src/java/main/org/apache/zookeeper/Login.java
+ * with the difference that refresh thread does not die.
+ */
+
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.callback.CallbackHandler;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.Shell;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.Subject;
+import java.util.Date;
+import java.util.Random;
+import java.util.Set;
+
+public class Login {
+    Logger LOG = Logger.getLogger(Login.class);
+    public CallbackHandler callbackHandler;
+
+    // Login will sleep until 80% of time from last refresh to
+    // ticket's expiry has been reached, at which time it will wake
+    // and try to renew the ticket.
+    private static final float TICKET_RENEW_WINDOW = 0.80f;
+
+    /**
+     * Percentage of random jitter added to the renewal time
+     */
+    private static final float TICKET_RENEW_JITTER = 0.05f;
+
+    // Regardless of TICKET_RENEW_WINDOW setting above and the ticket expiry time,
+    // thread will not sleep between refresh attempts any less than 1 minute (60*1000 milliseconds = 1 minute).
+    // Change the '1' to e.g. 5, to change this to 5 minutes.
+    private static final long MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L;
+
+    private Subject subject = null;
+    private Thread t = null;
+    private boolean isKrbTicket = false;
+    private boolean isUsingTicketCache = false;
+    private boolean isUsingKeytab = false;
+
+    /** Random number generator */
+    private static Random rng = new Random();
+
+    private LoginContext login = null;
+    private String loginContextName = null;
+    private String keytabFile = null;
+    private String principal = null;
+
+    private long lastLogin = 0;
+
+    /**
+     * Login constructor. The constructor starts the thread used
+     * to periodically re-login to the Kerberos Ticket Granting Server.
+     * @param loginContextName
+     *               name of section in JAAS file that will be use to login.
+     *               Passed as first param to javax.security.auth.login.LoginContext().
+     *
+     * @param callbackHandler
+     *               Passed as second param to javax.security.auth.login.LoginContext().
+     * @throws javax.security.auth.login.LoginException
+     *               Thrown if authentication fails.
+     */
+    public Login(final String loginContextName, CallbackHandler callbackHandler)
+        throws LoginException {
+        this.callbackHandler = callbackHandler;
+        login = login(loginContextName);
+        this.loginContextName = loginContextName;
+        subject = login.getSubject();
+        isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty();
+        AppConfigurationEntry entries[] = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
+        for (AppConfigurationEntry entry: entries) {
+            // there will only be a single entry, so this for() loop will only be iterated through once.
+            if (entry.getOptions().get("useTicketCache") != null) {
+                String val = (String)entry.getOptions().get("useTicketCache");
+                if (val.equals("true")) {
+                    isUsingTicketCache = true;
+                }
+            }
+            if (entry.getOptions().get("keyTab") != null) {
+                keytabFile = (String)entry.getOptions().get("keyTab");
+                isUsingKeytab = true;
+            }
+            if (entry.getOptions().get("principal") != null) {
+                principal = (String)entry.getOptions().get("principal");
+            }
+            break;
+        }
+
+        if (!isKrbTicket) {
+            // if no TGT, do not bother with ticket management.
+            return;
+        }
+
+        // Refresh the Ticket Granting Ticket (TGT) periodically. How often to refresh is determined by the
+        // TGT's existing expiry date and the configured MIN_TIME_BEFORE_RELOGIN. For testing and development,
+        // you can decrease the interval of expiration of tickets (for example, to 3 minutes) by running :
+        //  "modprinc -maxlife 3mins <principal>" in kadmin.
+        t = new Thread(new Runnable() {
+            public void run() {
+                LOG.info("TGT refresh thread started.");
+                while (true) {  // renewal thread's main loop. if it exits from here, thread will exit.
+                    KerberosTicket tgt = getTGT();
+                    long now = System.currentTimeMillis();
+                    long nextRefresh;
+                    Date nextRefreshDate;
+                    if (tgt == null) {
+                        nextRefresh = now + MIN_TIME_BEFORE_RELOGIN;
+                        nextRefreshDate = new Date(nextRefresh);
+                        LOG.warn("No TGT found: will try again at " + nextRefreshDate);
+                    } else {
+                        nextRefresh = getRefreshTime(tgt);
+                        long expiry = tgt.getEndTime().getTime();
+                        Date expiryDate = new Date(expiry);
+                        if ((isUsingTicketCache) && (tgt.getEndTime().equals(tgt.getRenewTill()))) {
+                            LOG.error("The TGT cannot be renewed beyond the next expiry date: " + expiryDate + "." +
+                                "This process will not be able to authenticate new SASL connections after that " +
+                                "time (for example, it will not be authenticate a new connection with a Zookeeper " +
+                                "Quorum member).  Ask your system administrator to either increase the " +
+                                "'renew until' time by doing : 'modprinc -maxrenewlife " + principal + "' within " +
+                                "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's " +
+                                "expiry cannot be further extended by refreshing, exiting refresh thread now.");
+                            return;
+                        }
+                        // determine how long to sleep from looking at ticket's expiry.
+                        // We should not allow the ticket to expire, but we should take into consideration
+                        // MIN_TIME_BEFORE_RELOGIN. Will not sleep less than MIN_TIME_BEFORE_RELOGIN, unless doing so
+                        // would cause ticket expiration.
+                        if ((nextRefresh > expiry) ||
+                            ((now + MIN_TIME_BEFORE_RELOGIN) > expiry)) {
+                            // expiry is before next scheduled refresh).
+                            nextRefresh = now;
+                        } else {
+                            if (nextRefresh < (now + MIN_TIME_BEFORE_RELOGIN)) {
+                                // next scheduled refresh is sooner than (now + MIN_TIME_BEFORE_LOGIN).
+                                Date until = new Date(nextRefresh);
+                                Date newuntil = new Date(now + MIN_TIME_BEFORE_RELOGIN);
+                                LOG.warn("TGT refresh thread time adjusted from : " + until + " to : " + newuntil + " since "
+                                    + "the former is sooner than the minimum refresh interval ("
+                                    + MIN_TIME_BEFORE_RELOGIN / 1000 + " seconds) from now.");
+                            }
+                            nextRefresh = Math.max(nextRefresh, now + MIN_TIME_BEFORE_RELOGIN);
+                        }
+                    }
+                    if (tgt != null && now > tgt.getEndTime().getTime()) {
+                        if ((now - tgt.getEndTime().getTime()) < ( 10 * MIN_TIME_BEFORE_RELOGIN)) {
+                            Date until = new Date(now + MIN_TIME_BEFORE_RELOGIN);
+                            LOG.info("TGT already expired but giving additional 10 minutes past TGT expiry, refresh sleeping until: " + until.toString());
+                            try {
+                                Thread.sleep(MIN_TIME_BEFORE_RELOGIN);
+                            } catch (InterruptedException ie) {
+                                LOG.warn("TGT renewal thread has been interrupted and will exit.");
+                                return;
+                            }
+                        } else {
+                            LOG.error("nextRefresh:" + new Date(nextRefresh) + " is in the past: exiting refresh thread. Check"
+                                + " clock sync between this host and KDC - (KDC's clock is likely ahead of this host)."
+                                + " Manual intervention will be required for this client to successfully authenticate."
+                                + " Exiting worker!.");
+                            Runtime.getRuntime().exit(-3);
+                        }
+                    } else if (now < nextRefresh) {
+                        Date until = new Date(nextRefresh);
+                        LOG.info("TGT refresh sleeping until: " + until.toString());
+                        try {
+                            Thread.sleep(nextRefresh - now);
+                        } catch (InterruptedException ie) {
+                            LOG.warn("TGT renewal thread has been interrupted and will exit.");
+                            return;
+                        }
+                    }
+
+                    if (isUsingTicketCache) {
+                        String cmd = "/usr/bin/kinit";
+                        if (System.getProperty("zookeeper.kinit") != null) {
+                            cmd = System.getProperty("zookeeper.kinit");
+                        }
+                        String kinitArgs = "-R";
+                        int retry = 1;
+                        while (retry >= 0) {
+                            try {
+                                LOG.debug("running ticket cache refresh command: " + cmd + " " + kinitArgs);
+                                Shell.execCommand(cmd, kinitArgs);
+                                break;
+                            } catch (Exception e) {
+                                if (retry > 0) {
+                                    --retry;
+                                    // sleep for 10 seconds
+                                    try {
+                                        Thread.sleep(10 * 1000);
+                                    } catch (InterruptedException ie) {
+                                        LOG.error("Interrupted while renewing TGT, exiting Login thread");
+                                        return;
+                                    }
+                                } else {
+                                    LOG.warn("Could not renew TGT due to problem running shell command: '" + cmd
+                                        + " " + kinitArgs + "'" + "; exception was:" + e + ". Exiting refresh thread.",e);
+                                    return;
+                                }
+                            }
+                        }
+                    }
+                    try {
+                        int retry = 1;
+                        while (retry >= 0) {
+                            try {
+                                reLogin();
+                                break;
+                            } catch (LoginException le) {
+                                if (retry > 0) {
+                                    --retry;
+                                    // sleep for 10 seconds.
+                                    try {
+                                        Thread.sleep(10 * 1000);
+                                    } catch (InterruptedException e) {
+                                        LOG.error("Interrupted during login retry after LoginException:", le);
+                                        throw le;
+                                    }
+                                } else {
+                                    LOG.error("Could not refresh TGT for principal: " + principal + ".", le);
+                                }
+                            }
+                        }
+                    } catch (LoginException le) {
+                        LOG.error("Failed to refresh TGT: refresh thread exiting now.",le);
+                        break;
+                    }
+                }
+            }
+        });
+        t.setName("Refresh-TGT");
+        t.setDaemon(true);
+    }
+
+    public void startThreadIfNeeded() {
+        // thread object 't' will be null if a refresh thread is not needed.
+        if (t != null) {
+            t.start();
+        }
+    }
+
+    public void shutdown() {
+        if ((t != null) && (t.isAlive())) {
+            t.interrupt();
+            try {
+                t.join();
+            } catch (InterruptedException e) {
+                LOG.warn("error while waiting for Login thread to shutdown: " + e);
+            }
+        }
+    }
+
+    public Subject getSubject() {
+        return subject;
+    }
+
+    public String getLoginContextName() {
+        return loginContextName;
+    }
+
+    private synchronized LoginContext login(final String loginContextName) throws LoginException {
+        if (loginContextName == null) {
+            throw new LoginException("loginContext name (JAAS file section header) was null. " +
+                "Please check your java.security.login.auth.config (=" +
+                System.getProperty("java.security.login.auth.config") +
+                ") and your " + ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY + "(=" +
+                System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client") + ")");
+        }
+        LoginContext loginContext = new LoginContext(loginContextName,callbackHandler);
+        loginContext.login();
+        LOG.info("successfully logged in.");
+        return loginContext;
+    }
+
+    // c.f. org.apache.hadoop.security.UserGroupInformation.
+    private long getRefreshTime(KerberosTicket tgt) {
+        long start = tgt.getStartTime().getTime();
+        long expires = tgt.getEndTime().getTime();
+        LOG.info("TGT valid starting at:        " + tgt.getStartTime().toString());
+        LOG.info("TGT expires:                  " + tgt.getEndTime().toString());
+        long proposedRefresh = start + (long) ((expires - start) *
+            (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble())));
+        if (proposedRefresh > expires) {
+            // proposedRefresh is too far in the future: it's after ticket expires: simply return now.
+            return System.currentTimeMillis();
+        }
+        else {
+            return proposedRefresh;
+        }
+    }
+
+    private synchronized KerberosTicket getTGT() {
+        Set<KerberosTicket> tickets = subject.getPrivateCredentials(KerberosTicket.class);
+        for(KerberosTicket ticket: tickets) {
+            KerberosPrincipal server = ticket.getServer();
+            if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) {
+                LOG.debug("Found tgt " + ticket + ".");
+                return ticket;
+            }
+        }
+        return null;
+    }
+
+    private void sleepUntilSufficientTimeElapsed() {
+        long now = System.currentTimeMillis();
+        if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN ) {
+            LOG.warn("Not attempting to re-login since the last re-login was " +
+                "attempted less than " + (MIN_TIME_BEFORE_RELOGIN/1000) + " seconds"+
+                " before.");
+            try {
+                Thread.sleep(MIN_TIME_BEFORE_RELOGIN - (now - getLastLogin()));
+            } catch (InterruptedException e) {
+                LOG.warn("TGT renewal thread has been interrupted and will exit.");
+                Runtime.getRuntime().exit(-2);
+            }
+        }
+        // register most recent relogin attempt
+        setLastLogin(System.currentTimeMillis());
+    }
+
+    /**
+     * Returns login object
+     * @return login
+     */
+    private LoginContext getLogin() {
+        return login;
+    }
+
+    /**
+     * Set the login object
+     * @param login
+     */
+    private void setLogin(LoginContext login) {
+        this.login = login;
+    }
+
+    /**
+     * Set the last login time.
+     * @param time the number of milliseconds since the beginning of time
+     */
+    private void setLastLogin(long time) {
+        lastLogin = time;
+    }
+
+    /**
+     * Get the time of the last login.
+     * @return the number of milliseconds since the beginning of time.
+     */
+    private long getLastLogin() {
+        return lastLogin;
+    }
+
+    /**
+     * Re-login a principal. This method assumes that {@link #login(String)} has happened already.
+     * @throws javax.security.auth.login.LoginException on a failure
+     */
+    // c.f. HADOOP-6559
+    private synchronized void reLogin()
+        throws LoginException {
+        if (!isKrbTicket) {
+            return;
+        }
+        LoginContext login = getLogin();
+        if (login  == null) {
+            throw new LoginException("login must be done first");
+        }
+        sleepUntilSufficientTimeElapsed();
+        LOG.info("Initiating logout for " + principal);
+        synchronized (Login.class) {
+            //clear up the kerberos state. But the tokens are not cleared! As per
+            //the Java kerberos login module code, only the kerberos credentials
+            //are cleared
+            login.logout();
+            //login and also update the subject field of this instance to
+            //have the new credentials (pass it to the LoginContext constructor)
+            login = new LoginContext(loginContextName, getSubject());
+            LOG.info("Initiating re-login for " + principal);
+            login.login();
+            setLogin(login);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java
new file mode 100644
index 0000000..1c5dd9d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.apache.storm.messaging.TaskMessage;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferOutputStream;
+import org.jboss.netty.buffer.ChannelBuffers;
+
+import java.util.ArrayList;
+
+class MessageBatch {
+    private int buffer_size;
+    private ArrayList<TaskMessage> msgs;
+    private int encoded_length;
+
+    MessageBatch(int buffer_size) {
+        this.buffer_size = buffer_size;
+        msgs = new ArrayList<>();
+        encoded_length = ControlMessage.EOB_MESSAGE.encodeLength();
+    }
+
+    void add(TaskMessage msg) {
+        if (msg == null)
+            throw new RuntimeException("null object forbidden in message batch");
+
+        msgs.add(msg);
+        encoded_length += msgEncodeLength(msg);
+    }
+
+
+    private int msgEncodeLength(TaskMessage taskMsg) {
+        if (taskMsg == null) return 0;
+
+        int size = 6; //INT + SHORT
+        if (taskMsg.message() != null) 
+            size += taskMsg.message().length;
+        return size;
+    }
+
+    /**
+     * @return true if this batch used up allowed buffer size
+     */
+    boolean isFull() {
+        return encoded_length >= buffer_size;
+    }
+
+    /**
+     * @return true if this batch doesn't have any messages
+     */
+    boolean isEmpty() {
+        return msgs.isEmpty();
+    }
+
+    /**
+     * @return number of msgs in this batch
+     */
+    int size() {
+        return msgs.size();
+    }
+
+    /**
+     * create a buffer containing the encoding of this batch
+     */
+    ChannelBuffer buffer() throws Exception {
+        ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encoded_length));
+        
+        for (TaskMessage msg : msgs) {
+            writeTaskMessage(bout, msg);
+        }
+
+        //add a END_OF_BATCH indicator
+        ControlMessage.EOB_MESSAGE.write(bout);
+
+        bout.close();
+
+        return bout.buffer();
+    }
+
+    /**
+     * write a TaskMessage into a stream
+     *
+     * Each TaskMessage is encoded as:
+     *  task ... short(2)
+     *  len ... int(4)
+     *  payload ... byte[]     *  
+     */
+    private void writeTaskMessage(ChannelBufferOutputStream bout, TaskMessage message) throws Exception {
+        int payload_len = 0;
+        if (message.message() != null)
+            payload_len =  message.message().length;
+
+        int task_id = message.task();
+        if (task_id > Short.MAX_VALUE)
+            throw new RuntimeException("Task ID should not exceed "+Short.MAX_VALUE);
+        
+        bout.writeShort((short)task_id);
+        bout.writeInt(payload_len);
+        if (payload_len >0)
+            bout.write(message.message());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBuffer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBuffer.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBuffer.java
new file mode 100644
index 0000000..4262f41
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBuffer.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.apache.storm.messaging.TaskMessage;
+
+/**
+ * Encapsulates the state used for batching up messages.
+ */
+public class MessageBuffer {
+    private final int mesageBatchSize;
+    private MessageBatch currentBatch;
+
+    public MessageBuffer(int mesageBatchSize){
+        this.mesageBatchSize = mesageBatchSize;
+        this.currentBatch = new MessageBatch(mesageBatchSize);
+    }
+
+    public synchronized MessageBatch add(TaskMessage msg){
+        currentBatch.add(msg);
+        if(currentBatch.isFull()){
+            MessageBatch ret = currentBatch;
+            currentBatch = new MessageBatch(mesageBatchSize);
+            return ret;
+        } else {
+            return null;
+        }
+    }
+
+    public synchronized boolean isEmpty() {
+        return currentBatch.isEmpty();
+    }
+
+    public synchronized MessageBatch drain() {
+        if(!currentBatch.isEmpty()) {
+            MessageBatch ret = currentBatch;
+            currentBatch = new MessageBatch(mesageBatchSize);
+            return ret;
+        } else {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java
new file mode 100644
index 0000000..9030424
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.storm.messaging.TaskMessage;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.frame.FrameDecoder;
+
+public class MessageDecoder extends FrameDecoder {    
+    /*
+     * Each ControlMessage is encoded as:
+     *  code (<0) ... short(2)
+     * Each TaskMessage is encoded as:
+     *  task (>=0) ... short(2)
+     *  len ... int(4)
+     *  payload ... byte[]     *  
+     */
+    protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buf) throws Exception {
+        // Make sure that we have received at least a short 
+        long available = buf.readableBytes();
+        if (available < 2) {
+            //need more data
+            return null;
+        }
+
+        List<Object> ret = new ArrayList<>();
+
+        // Use while loop, try to decode as more messages as possible in single call
+        while (available >= 2) {
+
+            // Mark the current buffer position before reading task/len field
+            // because the whole frame might not be in the buffer yet.
+            // We will reset the buffer position to the marked position if
+            // there's not enough bytes in the buffer.
+            buf.markReaderIndex();
+
+            // read the short field
+            short code = buf.readShort();
+            available -= 2;
+
+            // case 1: Control message
+            ControlMessage ctrl_msg = ControlMessage.mkMessage(code);
+            if (ctrl_msg != null) {
+
+                if (ctrl_msg == ControlMessage.EOB_MESSAGE) {
+                    continue;
+                } else {
+                    return ctrl_msg;
+                }
+            }
+            
+            //case 2: SaslTokenMessageRequest
+            if(code == SaslMessageToken.IDENTIFIER) {
+            	// Make sure that we have received at least an integer (length) 
+                if (buf.readableBytes() < 4) {
+                    //need more data
+                    buf.resetReaderIndex();
+                    return null;
+                }
+                
+                // Read the length field.
+                int length = buf.readInt();
+                if (length<=0) {
+                    return new SaslMessageToken(null);
+                }
+                
+                // Make sure if there's enough bytes in the buffer.
+                if (buf.readableBytes() < length) {
+                    // The whole bytes were not received yet - return null.
+                    buf.resetReaderIndex();
+                    return null;
+                }
+                
+                // There's enough bytes in the buffer. Read it.  
+                ChannelBuffer payload = buf.readBytes(length);
+                
+                // Successfully decoded a frame.
+                // Return a SaslTokenMessageRequest object
+                return new SaslMessageToken(payload.array());
+            }
+
+            // case 3: task Message
+
+            // Make sure that we have received at least an integer (length)
+            if (available < 4) {
+                // need more data
+                buf.resetReaderIndex();
+                break;
+            }
+
+            // Read the length field.
+            int length = buf.readInt();
+
+            available -= 4;
+
+            if (length <= 0) {
+                ret.add(new TaskMessage(code, null));
+                break;
+            }
+
+            // Make sure if there's enough bytes in the buffer.
+            if (available < length) {
+                // The whole bytes were not received yet - return null.
+                buf.resetReaderIndex();
+                break;
+            }
+            available -= length;
+
+            // There's enough bytes in the buffer. Read it.
+            ChannelBuffer payload = buf.readBytes(length);
+
+
+            // Successfully decoded a frame.
+            // Return a TaskMessage object
+            ret.add(new TaskMessage(code, payload.array()));
+        }
+
+        if (ret.size() == 0) {
+            return null;
+        } else {
+            return ret;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageEncoder.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageEncoder.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageEncoder.java
new file mode 100644
index 0000000..0e9fc98
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageEncoder.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.oneone.OneToOneEncoder;
+
+public class MessageEncoder extends OneToOneEncoder {    
+    @Override
+    protected Object encode(ChannelHandlerContext ctx, Channel channel, Object obj) throws Exception {
+        if (obj instanceof ControlMessage) {
+            return ((ControlMessage)obj).buffer();
+        }
+
+        if (obj instanceof MessageBatch) {
+            return ((MessageBatch)obj).buffer();
+        } 
+        
+        if (obj instanceof SaslMessageToken) {
+        	return ((SaslMessageToken)obj).buffer();
+        }
+        
+        throw new RuntimeException("Unsupported encoding of object of class "+obj.getClass().getName());
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyRenameThreadFactory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyRenameThreadFactory.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyRenameThreadFactory.java
new file mode 100644
index 0000000..e60c711
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyRenameThreadFactory.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jboss.netty.util.ThreadNameDeterminer;
+import org.jboss.netty.util.ThreadRenamingRunnable;
+
+public class NettyRenameThreadFactory  implements ThreadFactory {
+    
+    static {
+      //Rename Netty threads
+      ThreadRenamingRunnable.setThreadNameDeterminer(ThreadNameDeterminer.CURRENT);
+    }
+  
+    final ThreadGroup group;
+    final AtomicInteger index = new AtomicInteger(1);
+    final String name;
+    static final NettyUncaughtExceptionHandler uncaughtExceptionHandler = new NettyUncaughtExceptionHandler();
+
+    public NettyRenameThreadFactory(String name) {
+        SecurityManager s = System.getSecurityManager();
+        group = (s != null)? s.getThreadGroup() :
+                             Thread.currentThread().getThreadGroup();
+        this.name = name;
+    }
+
+    public Thread newThread(Runnable r) {
+        Thread t = new Thread(group, r, name + "-" + index.getAndIncrement(), 0);
+        if (t.isDaemon()) {
+            t.setDaemon(false);
+        }
+        if (t.getPriority() != Thread.NORM_PRIORITY) {
+            t.setPriority(Thread.NORM_PRIORITY);
+        }
+        t.setUncaughtExceptionHandler(uncaughtExceptionHandler);
+        return t;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyUncaughtExceptionHandler.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyUncaughtExceptionHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyUncaughtExceptionHandler.java
new file mode 100644
index 0000000..fd48bdc
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/NettyUncaughtExceptionHandler.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NettyUncaughtExceptionHandler implements Thread.UncaughtExceptionHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(NettyUncaughtExceptionHandler.class);
+    @Override
+    public void uncaughtException(Thread t, Throwable e) {
+        try {
+            Utils.handleUncaughtException(e);
+        } catch (Error error) {
+            LOG.info("Received error in netty thread.. terminating server...");
+            Runtime.getRuntime().exit(1);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java
new file mode 100644
index 0000000..ccac70f
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.io.IOException;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferOutputStream;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Send and receive SASL tokens.
+ */
+public class SaslMessageToken implements INettySerializable {
+    public static final short IDENTIFIER = -500;
+
+    /** Class logger */
+    private static final Logger LOG = LoggerFactory
+            .getLogger(SaslMessageToken.class);
+
+    /** Used for client or server's token to send or receive from each other. */
+    private byte[] token;
+
+    /**
+     * Constructor used for reflection only.
+     */
+    public SaslMessageToken() {
+    }
+
+    /**
+     * Constructor used to send request.
+     * 
+     * @param token
+     *            the SASL token, generated by a SaslClient or SaslServer.
+     */
+    public SaslMessageToken(byte[] token) {
+        this.token = token;
+    }
+
+    /**
+     * Read accessor for SASL token
+     * 
+     * @return saslToken SASL token
+     */
+    public byte[] getSaslToken() {
+        return token;
+    }
+
+    /**
+     * Write accessor for SASL token
+     * 
+     * @param token
+     *            SASL token
+     */
+    public void setSaslToken(byte[] token) {
+        this.token = token;
+    }
+
+    public int encodeLength() {
+        return 2 + 4 + token.length;
+    }
+
+    /**
+     * encode the current SaslToken Message into a channel buffer
+     * SaslTokenMessageRequest is encoded as: identifier .... short(2)
+     * payload length .... int payload .... byte[]
+     * 
+     * @throws IOException
+     */
+    public ChannelBuffer buffer() throws IOException {
+        ChannelBufferOutputStream bout = new ChannelBufferOutputStream(
+                ChannelBuffers.directBuffer(encodeLength()));
+        int payload_len = 0;
+        if (token != null)
+            payload_len = token.length;
+
+        bout.writeShort(IDENTIFIER);
+        bout.writeInt(payload_len);
+
+        if (payload_len > 0) {
+            bout.write(token);
+        }
+        bout.close();
+        return bout.buffer();
+    }
+    
+    public static SaslMessageToken read(byte[] serial) {
+        ChannelBuffer sm_buffer = ChannelBuffers.copiedBuffer(serial);
+        short identifier = sm_buffer.readShort();
+        int payload_len = sm_buffer.readInt();
+        if(identifier != IDENTIFIER) {
+            return null;
+        }
+        byte token[] = new byte[payload_len];
+        sm_buffer.readBytes(token, 0, payload_len);
+        return new SaslMessageToken(token);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java
new file mode 100644
index 0000000..b24e7d6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.io.IOException;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.RealmCallback;
+import javax.security.sasl.RealmChoiceCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements SASL logic for storm worker client processes.
+ */
+public class SaslNettyClient {
+
+    private static final Logger LOG = LoggerFactory
+            .getLogger(SaslNettyClient.class);
+
+    /**
+     * Used to respond to server's counterpart, SaslServer with SASL tokens
+     * represented as byte arrays.
+     */
+    private SaslClient saslClient;
+
+    /**
+     * Create a SaslNettyClient for authentication with servers.
+     */
+    public SaslNettyClient(String topologyName, byte[] token) {
+        try {
+            LOG.debug("SaslNettyClient: Creating SASL {} client to authenticate to server ",
+                      SaslUtils.AUTH_DIGEST_MD5);
+
+            saslClient = Sasl.createSaslClient(
+                    new String[] { SaslUtils.AUTH_DIGEST_MD5 }, null, null,
+                    SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(),
+                    new SaslClientCallbackHandler(topologyName, token));
+
+        } catch (IOException e) {
+            LOG.error("SaslNettyClient: Could not obtain topology token for Netty "
+                    + "Client to use to authenticate with a Netty Server.");
+            saslClient = null;
+        }
+    }
+
+    public boolean isComplete() {
+        return saslClient.isComplete();
+    }
+
+    /**
+     * Respond to server's SASL token.
+     * 
+     * @param saslTokenMessage
+     *            contains server's SASL token
+     * @return client's response SASL token
+     */
+    public byte[] saslResponse(SaslMessageToken saslTokenMessage) {
+        try {
+            return saslClient.evaluateChallenge(saslTokenMessage.getSaslToken());
+        } catch (SaslException e) {
+            LOG.error(
+                    "saslResponse: Failed to respond to SASL server's token:",
+                    e);
+            return null;
+        }
+    }
+
+    /**
+     * Implementation of javax.security.auth.callback.CallbackHandler that works
+     * with Storm topology tokens.
+     */
+    private static class SaslClientCallbackHandler implements CallbackHandler {
+        /** Generated username contained in TopologyToken */
+        private final String userName;
+        /** Generated password contained in TopologyToken */
+        private final char[] userPassword;
+
+        /**
+         * Set private members using topology token.
+         */
+        public SaslClientCallbackHandler(String topologyToken, byte[] token) {
+            this.userName = SaslUtils
+                    .encodeIdentifier(topologyToken.getBytes());
+            this.userPassword = SaslUtils.encodePassword(token);
+        }
+
+        /**
+         * Implementation used to respond to SASL tokens from server.
+         * 
+         * @param callbacks
+         *            objects that indicate what credential information the
+         *            server's SaslServer requires from the client.
+         * @throws UnsupportedCallbackException
+         */
+        public void handle(Callback[] callbacks)
+                throws UnsupportedCallbackException {
+            NameCallback nc = null;
+            PasswordCallback pc = null;
+            RealmCallback rc = null;
+            for (Callback callback : callbacks) {
+                if (callback instanceof RealmChoiceCallback) {
+                    continue;
+                } else if (callback instanceof NameCallback) {
+                    nc = (NameCallback) callback;
+                } else if (callback instanceof PasswordCallback) {
+                    pc = (PasswordCallback) callback;
+                } else if (callback instanceof RealmCallback) {
+                    rc = (RealmCallback) callback;
+                } else {
+                    throw new UnsupportedCallbackException(callback,
+                            "handle: Unrecognized SASL client callback");
+                }
+            }
+            if (nc != null) {
+                LOG.debug("handle: SASL client callback: setting username: {}",
+                          userName);
+                nc.setName(userName);
+            }
+            if (pc != null) {
+                LOG.debug("handle: SASL client callback: setting userPassword");
+                pc.setPassword(userPassword);
+            }
+            if (rc != null) {
+                LOG.debug("handle: SASL client callback: setting realm: {}",
+                        rc.getDefaultText());
+                rc.setText(rc.getDefaultText());
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClientState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClientState.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClientState.java
new file mode 100644
index 0000000..3a7d6a2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClientState.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelLocal;
+
+final class SaslNettyClientState {
+
+	public static final ChannelLocal<SaslNettyClient> getSaslNettyClient = new ChannelLocal<SaslNettyClient>() {
+		protected SaslNettyClient initialValue(Channel channel) {
+			return null;
+		}
+	};
+
+}


[23/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/KeyNotFoundException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/KeyNotFoundException.java b/storm-client/src/jvm/org/apache/storm/generated/KeyNotFoundException.java
new file mode 100644
index 0000000..d55564c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/KeyNotFoundException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class KeyNotFoundException extends TException implements org.apache.thrift.TBase<KeyNotFoundException, KeyNotFoundException._Fields>, java.io.Serializable, Cloneable, Comparable<KeyNotFoundException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KeyNotFoundException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new KeyNotFoundExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new KeyNotFoundExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KeyNotFoundException.class, metaDataMap);
+  }
+
+  public KeyNotFoundException() {
+  }
+
+  public KeyNotFoundException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public KeyNotFoundException(KeyNotFoundException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public KeyNotFoundException deepCopy() {
+    return new KeyNotFoundException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof KeyNotFoundException)
+      return this.equals((KeyNotFoundException)that);
+    return false;
+  }
+
+  public boolean equals(KeyNotFoundException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(KeyNotFoundException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("KeyNotFoundException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class KeyNotFoundExceptionStandardSchemeFactory implements SchemeFactory {
+    public KeyNotFoundExceptionStandardScheme getScheme() {
+      return new KeyNotFoundExceptionStandardScheme();
+    }
+  }
+
+  private static class KeyNotFoundExceptionStandardScheme extends StandardScheme<KeyNotFoundException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class KeyNotFoundExceptionTupleSchemeFactory implements SchemeFactory {
+    public KeyNotFoundExceptionTupleScheme getScheme() {
+      return new KeyNotFoundExceptionTupleScheme();
+    }
+  }
+
+  private static class KeyNotFoundExceptionTupleScheme extends TupleScheme<KeyNotFoundException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/KillOptions.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/KillOptions.java b/storm-client/src/jvm/org/apache/storm/generated/KillOptions.java
new file mode 100644
index 0000000..f2ad8e2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/KillOptions.java
@@ -0,0 +1,407 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class KillOptions implements org.apache.thrift.TBase<KillOptions, KillOptions._Fields>, java.io.Serializable, Cloneable, Comparable<KillOptions> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KillOptions");
+
+  private static final org.apache.thrift.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("wait_secs", org.apache.thrift.protocol.TType.I32, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new KillOptionsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new KillOptionsTupleSchemeFactory());
+  }
+
+  private int wait_secs; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    WAIT_SECS((short)1, "wait_secs");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // WAIT_SECS
+          return WAIT_SECS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __WAIT_SECS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.WAIT_SECS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift.meta_data.FieldMetaData("wait_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KillOptions.class, metaDataMap);
+  }
+
+  public KillOptions() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public KillOptions(KillOptions other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.wait_secs = other.wait_secs;
+  }
+
+  public KillOptions deepCopy() {
+    return new KillOptions(this);
+  }
+
+  @Override
+  public void clear() {
+    set_wait_secs_isSet(false);
+    this.wait_secs = 0;
+  }
+
+  public int get_wait_secs() {
+    return this.wait_secs;
+  }
+
+  public void set_wait_secs(int wait_secs) {
+    this.wait_secs = wait_secs;
+    set_wait_secs_isSet(true);
+  }
+
+  public void unset_wait_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WAIT_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_wait_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __WAIT_SECS_ISSET_ID);
+  }
+
+  public void set_wait_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WAIT_SECS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case WAIT_SECS:
+      if (value == null) {
+        unset_wait_secs();
+      } else {
+        set_wait_secs((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case WAIT_SECS:
+      return get_wait_secs();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case WAIT_SECS:
+      return is_set_wait_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof KillOptions)
+      return this.equals((KillOptions)that);
+    return false;
+  }
+
+  public boolean equals(KillOptions that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_wait_secs = true && this.is_set_wait_secs();
+    boolean that_present_wait_secs = true && that.is_set_wait_secs();
+    if (this_present_wait_secs || that_present_wait_secs) {
+      if (!(this_present_wait_secs && that_present_wait_secs))
+        return false;
+      if (this.wait_secs != that.wait_secs)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_wait_secs = true && (is_set_wait_secs());
+    list.add(present_wait_secs);
+    if (present_wait_secs)
+      list.add(wait_secs);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(KillOptions other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(other.is_set_wait_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_wait_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.wait_secs, other.wait_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("KillOptions(");
+    boolean first = true;
+
+    if (is_set_wait_secs()) {
+      sb.append("wait_secs:");
+      sb.append(this.wait_secs);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class KillOptionsStandardSchemeFactory implements SchemeFactory {
+    public KillOptionsStandardScheme getScheme() {
+      return new KillOptionsStandardScheme();
+    }
+  }
+
+  private static class KillOptionsStandardScheme extends StandardScheme<KillOptions> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, KillOptions struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // WAIT_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.wait_secs = iprot.readI32();
+              struct.set_wait_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, KillOptions struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_wait_secs()) {
+        oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC);
+        oprot.writeI32(struct.wait_secs);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class KillOptionsTupleSchemeFactory implements SchemeFactory {
+    public KillOptionsTupleScheme getScheme() {
+      return new KillOptionsTupleScheme();
+    }
+  }
+
+  private static class KillOptionsTupleScheme extends TupleScheme<KillOptions> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, KillOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_wait_secs()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_wait_secs()) {
+        oprot.writeI32(struct.wait_secs);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, KillOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.wait_secs = iprot.readI32();
+        struct.set_wait_secs_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java b/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java
new file mode 100644
index 0000000..b4795a5
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java
@@ -0,0 +1,458 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWorkers, LSApprovedWorkers._Fields>, java.io.Serializable, Cloneable, Comparable<LSApprovedWorkers> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSApprovedWorkers");
+
+  private static final org.apache.thrift.protocol.TField APPROVED_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("approved_workers", org.apache.thrift.protocol.TType.MAP, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSApprovedWorkersStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSApprovedWorkersTupleSchemeFactory());
+  }
+
+  private Map<String,Integer> approved_workers; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    APPROVED_WORKERS((short)1, "approved_workers");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // APPROVED_WORKERS
+          return APPROVED_WORKERS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.APPROVED_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("approved_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSApprovedWorkers.class, metaDataMap);
+  }
+
+  public LSApprovedWorkers() {
+  }
+
+  public LSApprovedWorkers(
+    Map<String,Integer> approved_workers)
+  {
+    this();
+    this.approved_workers = approved_workers;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSApprovedWorkers(LSApprovedWorkers other) {
+    if (other.is_set_approved_workers()) {
+      Map<String,Integer> __this__approved_workers = new HashMap<String,Integer>(other.approved_workers);
+      this.approved_workers = __this__approved_workers;
+    }
+  }
+
+  public LSApprovedWorkers deepCopy() {
+    return new LSApprovedWorkers(this);
+  }
+
+  @Override
+  public void clear() {
+    this.approved_workers = null;
+  }
+
+  public int get_approved_workers_size() {
+    return (this.approved_workers == null) ? 0 : this.approved_workers.size();
+  }
+
+  public void put_to_approved_workers(String key, int val) {
+    if (this.approved_workers == null) {
+      this.approved_workers = new HashMap<String,Integer>();
+    }
+    this.approved_workers.put(key, val);
+  }
+
+  public Map<String,Integer> get_approved_workers() {
+    return this.approved_workers;
+  }
+
+  public void set_approved_workers(Map<String,Integer> approved_workers) {
+    this.approved_workers = approved_workers;
+  }
+
+  public void unset_approved_workers() {
+    this.approved_workers = null;
+  }
+
+  /** Returns true if field approved_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_approved_workers() {
+    return this.approved_workers != null;
+  }
+
+  public void set_approved_workers_isSet(boolean value) {
+    if (!value) {
+      this.approved_workers = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case APPROVED_WORKERS:
+      if (value == null) {
+        unset_approved_workers();
+      } else {
+        set_approved_workers((Map<String,Integer>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case APPROVED_WORKERS:
+      return get_approved_workers();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case APPROVED_WORKERS:
+      return is_set_approved_workers();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSApprovedWorkers)
+      return this.equals((LSApprovedWorkers)that);
+    return false;
+  }
+
+  public boolean equals(LSApprovedWorkers that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_approved_workers = true && this.is_set_approved_workers();
+    boolean that_present_approved_workers = true && that.is_set_approved_workers();
+    if (this_present_approved_workers || that_present_approved_workers) {
+      if (!(this_present_approved_workers && that_present_approved_workers))
+        return false;
+      if (!this.approved_workers.equals(that.approved_workers))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_approved_workers = true && (is_set_approved_workers());
+    list.add(present_approved_workers);
+    if (present_approved_workers)
+      list.add(approved_workers);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSApprovedWorkers other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_approved_workers()).compareTo(other.is_set_approved_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_approved_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.approved_workers, other.approved_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSApprovedWorkers(");
+    boolean first = true;
+
+    sb.append("approved_workers:");
+    if (this.approved_workers == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.approved_workers);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_approved_workers()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'approved_workers' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSApprovedWorkersStandardSchemeFactory implements SchemeFactory {
+    public LSApprovedWorkersStandardScheme getScheme() {
+      return new LSApprovedWorkersStandardScheme();
+    }
+  }
+
+  private static class LSApprovedWorkersStandardScheme extends StandardScheme<LSApprovedWorkers> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // APPROVED_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map722 = iprot.readMapBegin();
+                struct.approved_workers = new HashMap<String,Integer>(2*_map722.size);
+                String _key723;
+                int _val724;
+                for (int _i725 = 0; _i725 < _map722.size; ++_i725)
+                {
+                  _key723 = iprot.readString();
+                  _val724 = iprot.readI32();
+                  struct.approved_workers.put(_key723, _val724);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_approved_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.approved_workers != null) {
+        oprot.writeFieldBegin(APPROVED_WORKERS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.approved_workers.size()));
+          for (Map.Entry<String, Integer> _iter726 : struct.approved_workers.entrySet())
+          {
+            oprot.writeString(_iter726.getKey());
+            oprot.writeI32(_iter726.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSApprovedWorkersTupleSchemeFactory implements SchemeFactory {
+    public LSApprovedWorkersTupleScheme getScheme() {
+      return new LSApprovedWorkersTupleScheme();
+    }
+  }
+
+  private static class LSApprovedWorkersTupleScheme extends TupleScheme<LSApprovedWorkers> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.approved_workers.size());
+        for (Map.Entry<String, Integer> _iter727 : struct.approved_workers.entrySet())
+        {
+          oprot.writeString(_iter727.getKey());
+          oprot.writeI32(_iter727.getValue());
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map728 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.approved_workers = new HashMap<String,Integer>(2*_map728.size);
+        String _key729;
+        int _val730;
+        for (int _i731 = 0; _i731 < _map728.size; ++_i731)
+        {
+          _key729 = iprot.readString();
+          _val730 = iprot.readI32();
+          struct.approved_workers.put(_key729, _val730);
+        }
+      }
+      struct.set_approved_workers_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java b/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java
new file mode 100644
index 0000000..b3194ad
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java
@@ -0,0 +1,471 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSupervisorAssignments, LSSupervisorAssignments._Fields>, java.io.Serializable, Cloneable, Comparable<LSSupervisorAssignments> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorAssignments");
+
+  private static final org.apache.thrift.protocol.TField ASSIGNMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("assignments", org.apache.thrift.protocol.TType.MAP, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSSupervisorAssignmentsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSSupervisorAssignmentsTupleSchemeFactory());
+  }
+
+  private Map<Integer,LocalAssignment> assignments; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ASSIGNMENTS((short)1, "assignments");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ASSIGNMENTS
+          return ASSIGNMENTS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ASSIGNMENTS, new org.apache.thrift.meta_data.FieldMetaData("assignments", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LocalAssignment.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSSupervisorAssignments.class, metaDataMap);
+  }
+
+  public LSSupervisorAssignments() {
+  }
+
+  public LSSupervisorAssignments(
+    Map<Integer,LocalAssignment> assignments)
+  {
+    this();
+    this.assignments = assignments;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSSupervisorAssignments(LSSupervisorAssignments other) {
+    if (other.is_set_assignments()) {
+      Map<Integer,LocalAssignment> __this__assignments = new HashMap<Integer,LocalAssignment>(other.assignments.size());
+      for (Map.Entry<Integer, LocalAssignment> other_element : other.assignments.entrySet()) {
+
+        Integer other_element_key = other_element.getKey();
+        LocalAssignment other_element_value = other_element.getValue();
+
+        Integer __this__assignments_copy_key = other_element_key;
+
+        LocalAssignment __this__assignments_copy_value = new LocalAssignment(other_element_value);
+
+        __this__assignments.put(__this__assignments_copy_key, __this__assignments_copy_value);
+      }
+      this.assignments = __this__assignments;
+    }
+  }
+
+  public LSSupervisorAssignments deepCopy() {
+    return new LSSupervisorAssignments(this);
+  }
+
+  @Override
+  public void clear() {
+    this.assignments = null;
+  }
+
+  public int get_assignments_size() {
+    return (this.assignments == null) ? 0 : this.assignments.size();
+  }
+
+  public void put_to_assignments(int key, LocalAssignment val) {
+    if (this.assignments == null) {
+      this.assignments = new HashMap<Integer,LocalAssignment>();
+    }
+    this.assignments.put(key, val);
+  }
+
+  public Map<Integer,LocalAssignment> get_assignments() {
+    return this.assignments;
+  }
+
+  public void set_assignments(Map<Integer,LocalAssignment> assignments) {
+    this.assignments = assignments;
+  }
+
+  public void unset_assignments() {
+    this.assignments = null;
+  }
+
+  /** Returns true if field assignments is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assignments() {
+    return this.assignments != null;
+  }
+
+  public void set_assignments_isSet(boolean value) {
+    if (!value) {
+      this.assignments = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ASSIGNMENTS:
+      if (value == null) {
+        unset_assignments();
+      } else {
+        set_assignments((Map<Integer,LocalAssignment>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ASSIGNMENTS:
+      return get_assignments();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ASSIGNMENTS:
+      return is_set_assignments();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSSupervisorAssignments)
+      return this.equals((LSSupervisorAssignments)that);
+    return false;
+  }
+
+  public boolean equals(LSSupervisorAssignments that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_assignments = true && this.is_set_assignments();
+    boolean that_present_assignments = true && that.is_set_assignments();
+    if (this_present_assignments || that_present_assignments) {
+      if (!(this_present_assignments && that_present_assignments))
+        return false;
+      if (!this.assignments.equals(that.assignments))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_assignments = true && (is_set_assignments());
+    list.add(present_assignments);
+    if (present_assignments)
+      list.add(assignments);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSSupervisorAssignments other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_assignments()).compareTo(other.is_set_assignments());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assignments()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assignments, other.assignments);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSSupervisorAssignments(");
+    boolean first = true;
+
+    sb.append("assignments:");
+    if (this.assignments == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.assignments);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_assignments()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'assignments' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSSupervisorAssignmentsStandardSchemeFactory implements SchemeFactory {
+    public LSSupervisorAssignmentsStandardScheme getScheme() {
+      return new LSSupervisorAssignmentsStandardScheme();
+    }
+  }
+
+  private static class LSSupervisorAssignmentsStandardScheme extends StandardScheme<LSSupervisorAssignments> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ASSIGNMENTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map732 = iprot.readMapBegin();
+                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map732.size);
+                int _key733;
+                LocalAssignment _val734;
+                for (int _i735 = 0; _i735 < _map732.size; ++_i735)
+                {
+                  _key733 = iprot.readI32();
+                  _val734 = new LocalAssignment();
+                  _val734.read(iprot);
+                  struct.assignments.put(_key733, _val734);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_assignments_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.assignments != null) {
+        oprot.writeFieldBegin(ASSIGNMENTS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.assignments.size()));
+          for (Map.Entry<Integer, LocalAssignment> _iter736 : struct.assignments.entrySet())
+          {
+            oprot.writeI32(_iter736.getKey());
+            _iter736.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSSupervisorAssignmentsTupleSchemeFactory implements SchemeFactory {
+    public LSSupervisorAssignmentsTupleScheme getScheme() {
+      return new LSSupervisorAssignmentsTupleScheme();
+    }
+  }
+
+  private static class LSSupervisorAssignmentsTupleScheme extends TupleScheme<LSSupervisorAssignments> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.assignments.size());
+        for (Map.Entry<Integer, LocalAssignment> _iter737 : struct.assignments.entrySet())
+        {
+          oprot.writeI32(_iter737.getKey());
+          _iter737.getValue().write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map738 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map738.size);
+        int _key739;
+        LocalAssignment _val740;
+        for (int _i741 = 0; _i741 < _map738.size; ++_i741)
+        {
+          _key739 = iprot.readI32();
+          _val740 = new LocalAssignment();
+          _val740.read(iprot);
+          struct.assignments.put(_key739, _val740);
+        }
+      }
+      struct.set_assignments_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorId.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorId.java b/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorId.java
new file mode 100644
index 0000000..605b9b1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorId.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSSupervisorId implements org.apache.thrift.TBase<LSSupervisorId, LSSupervisorId._Fields>, java.io.Serializable, Cloneable, Comparable<LSSupervisorId> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorId");
+
+  private static final org.apache.thrift.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSSupervisorIdStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSSupervisorIdTupleSchemeFactory());
+  }
+
+  private String supervisor_id; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SUPERVISOR_ID((short)1, "supervisor_id");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SUPERVISOR_ID
+          return SUPERVISOR_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift.meta_data.FieldMetaData("supervisor_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSSupervisorId.class, metaDataMap);
+  }
+
+  public LSSupervisorId() {
+  }
+
+  public LSSupervisorId(
+    String supervisor_id)
+  {
+    this();
+    this.supervisor_id = supervisor_id;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSSupervisorId(LSSupervisorId other) {
+    if (other.is_set_supervisor_id()) {
+      this.supervisor_id = other.supervisor_id;
+    }
+  }
+
+  public LSSupervisorId deepCopy() {
+    return new LSSupervisorId(this);
+  }
+
+  @Override
+  public void clear() {
+    this.supervisor_id = null;
+  }
+
+  public String get_supervisor_id() {
+    return this.supervisor_id;
+  }
+
+  public void set_supervisor_id(String supervisor_id) {
+    this.supervisor_id = supervisor_id;
+  }
+
+  public void unset_supervisor_id() {
+    this.supervisor_id = null;
+  }
+
+  /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_supervisor_id() {
+    return this.supervisor_id != null;
+  }
+
+  public void set_supervisor_id_isSet(boolean value) {
+    if (!value) {
+      this.supervisor_id = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SUPERVISOR_ID:
+      if (value == null) {
+        unset_supervisor_id();
+      } else {
+        set_supervisor_id((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SUPERVISOR_ID:
+      return get_supervisor_id();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SUPERVISOR_ID:
+      return is_set_supervisor_id();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSSupervisorId)
+      return this.equals((LSSupervisorId)that);
+    return false;
+  }
+
+  public boolean equals(LSSupervisorId that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_supervisor_id = true && this.is_set_supervisor_id();
+    boolean that_present_supervisor_id = true && that.is_set_supervisor_id();
+    if (this_present_supervisor_id || that_present_supervisor_id) {
+      if (!(this_present_supervisor_id && that_present_supervisor_id))
+        return false;
+      if (!this.supervisor_id.equals(that.supervisor_id))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_supervisor_id = true && (is_set_supervisor_id());
+    list.add(present_supervisor_id);
+    if (present_supervisor_id)
+      list.add(supervisor_id);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSSupervisorId other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(other.is_set_supervisor_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_supervisor_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_id, other.supervisor_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSSupervisorId(");
+    boolean first = true;
+
+    sb.append("supervisor_id:");
+    if (this.supervisor_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.supervisor_id);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_supervisor_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor_id' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSSupervisorIdStandardSchemeFactory implements SchemeFactory {
+    public LSSupervisorIdStandardScheme getScheme() {
+      return new LSSupervisorIdStandardScheme();
+    }
+  }
+
+  private static class LSSupervisorIdStandardScheme extends StandardScheme<LSSupervisorId> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSSupervisorId struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SUPERVISOR_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.supervisor_id = iprot.readString();
+              struct.set_supervisor_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSSupervisorId struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.supervisor_id != null) {
+        oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC);
+        oprot.writeString(struct.supervisor_id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSSupervisorIdTupleSchemeFactory implements SchemeFactory {
+    public LSSupervisorIdTupleScheme getScheme() {
+      return new LSSupervisorIdTupleScheme();
+    }
+  }
+
+  private static class LSSupervisorIdTupleScheme extends TupleScheme<LSSupervisorId> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSSupervisorId struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.supervisor_id);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSSupervisorId struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.supervisor_id = iprot.readString();
+      struct.set_supervisor_id_isSet(true);
+    }
+  }
+
+}
+


[20/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LogLevelAction.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LogLevelAction.java b/storm-client/src/jvm/org/apache/storm/generated/LogLevelAction.java
new file mode 100644
index 0000000..1f0a41a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LogLevelAction.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum LogLevelAction implements org.apache.thrift.TEnum {
+  UNCHANGED(1),
+  UPDATE(2),
+  REMOVE(3);
+
+  private final int value;
+
+  private LogLevelAction(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static LogLevelAction findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return UNCHANGED;
+      case 2:
+        return UPDATE;
+      case 3:
+        return REMOVE;
+      default:
+        return null;
+    }
+  }
+}


[04/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/WorkerSummary.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerSummary.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerSummary.java
new file mode 100644
index 0000000..fff9373
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerSummary.java
@@ -0,0 +1,1880 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class WorkerSummary implements org.apache.thrift.TBase<WorkerSummary, WorkerSummary._Fields>, java.io.Serializable, Cloneable, Comparable<WorkerSummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerSummary");
+
+  private static final org.apache.thrift.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_name", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField COMPONENT_TO_NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("component_to_num_tasks", org.apache.thrift.protocol.TType.MAP, (short)7);
+  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)8);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)9);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WorkerSummaryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WorkerSummaryTupleSchemeFactory());
+  }
+
+  private String supervisor_id; // optional
+  private String host; // optional
+  private int port; // optional
+  private String topology_id; // optional
+  private String topology_name; // optional
+  private int num_executors; // optional
+  private Map<String,Long> component_to_num_tasks; // optional
+  private int time_secs; // optional
+  private int uptime_secs; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SUPERVISOR_ID((short)1, "supervisor_id"),
+    HOST((short)2, "host"),
+    PORT((short)3, "port"),
+    TOPOLOGY_ID((short)4, "topology_id"),
+    TOPOLOGY_NAME((short)5, "topology_name"),
+    NUM_EXECUTORS((short)6, "num_executors"),
+    COMPONENT_TO_NUM_TASKS((short)7, "component_to_num_tasks"),
+    TIME_SECS((short)8, "time_secs"),
+    UPTIME_SECS((short)9, "uptime_secs"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SUPERVISOR_ID
+          return SUPERVISOR_ID;
+        case 2: // HOST
+          return HOST;
+        case 3: // PORT
+          return PORT;
+        case 4: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 5: // TOPOLOGY_NAME
+          return TOPOLOGY_NAME;
+        case 6: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 7: // COMPONENT_TO_NUM_TASKS
+          return COMPONENT_TO_NUM_TASKS;
+        case 8: // TIME_SECS
+          return TIME_SECS;
+        case 9: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PORT_ISSET_ID = 0;
+  private static final int __NUM_EXECUTORS_ISSET_ID = 1;
+  private static final int __TIME_SECS_ISSET_ID = 2;
+  private static final int __UPTIME_SECS_ISSET_ID = 3;
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 4;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 5;
+  private static final int __REQUESTED_CPU_ISSET_ID = 6;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 7;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 8;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 9;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.SUPERVISOR_ID,_Fields.HOST,_Fields.PORT,_Fields.TOPOLOGY_ID,_Fields.TOPOLOGY_NAME,_Fields.NUM_EXECUTORS,_Fields.COMPONENT_TO_NUM_TASKS,_Fields.TIME_SECS,_Fields.UPTIME_SECS,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift.meta_data.FieldMetaData("supervisor_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topology_name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.COMPONENT_TO_NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("component_to_num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerSummary.class, metaDataMap);
+  }
+
+  public WorkerSummary() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WorkerSummary(WorkerSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_supervisor_id()) {
+      this.supervisor_id = other.supervisor_id;
+    }
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.port = other.port;
+    if (other.is_set_topology_id()) {
+      this.topology_id = other.topology_id;
+    }
+    if (other.is_set_topology_name()) {
+      this.topology_name = other.topology_name;
+    }
+    this.num_executors = other.num_executors;
+    if (other.is_set_component_to_num_tasks()) {
+      Map<String,Long> __this__component_to_num_tasks = new HashMap<String,Long>(other.component_to_num_tasks);
+      this.component_to_num_tasks = __this__component_to_num_tasks;
+    }
+    this.time_secs = other.time_secs;
+    this.uptime_secs = other.uptime_secs;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
+  }
+
+  public WorkerSummary deepCopy() {
+    return new WorkerSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.supervisor_id = null;
+    this.host = null;
+    set_port_isSet(false);
+    this.port = 0;
+    this.topology_id = null;
+    this.topology_name = null;
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    this.component_to_num_tasks = null;
+    set_time_secs_isSet(false);
+    this.time_secs = 0;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
+  }
+
+  public String get_supervisor_id() {
+    return this.supervisor_id;
+  }
+
+  public void set_supervisor_id(String supervisor_id) {
+    this.supervisor_id = supervisor_id;
+  }
+
+  public void unset_supervisor_id() {
+    this.supervisor_id = null;
+  }
+
+  /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_supervisor_id() {
+    return this.supervisor_id != null;
+  }
+
+  public void set_supervisor_id_isSet(boolean value) {
+    if (!value) {
+      this.supervisor_id = null;
+    }
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
+  }
+
+  public String get_topology_id() {
+    return this.topology_id;
+  }
+
+  public void set_topology_id(String topology_id) {
+    this.topology_id = topology_id;
+  }
+
+  public void unset_topology_id() {
+    this.topology_id = null;
+  }
+
+  /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_id() {
+    return this.topology_id != null;
+  }
+
+  public void set_topology_id_isSet(boolean value) {
+    if (!value) {
+      this.topology_id = null;
+    }
+  }
+
+  public String get_topology_name() {
+    return this.topology_name;
+  }
+
+  public void set_topology_name(String topology_name) {
+    this.topology_name = topology_name;
+  }
+
+  public void unset_topology_name() {
+    this.topology_name = null;
+  }
+
+  /** Returns true if field topology_name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_name() {
+    return this.topology_name != null;
+  }
+
+  public void set_topology_name_isSet(boolean value) {
+    if (!value) {
+      this.topology_name = null;
+    }
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public int get_component_to_num_tasks_size() {
+    return (this.component_to_num_tasks == null) ? 0 : this.component_to_num_tasks.size();
+  }
+
+  public void put_to_component_to_num_tasks(String key, long val) {
+    if (this.component_to_num_tasks == null) {
+      this.component_to_num_tasks = new HashMap<String,Long>();
+    }
+    this.component_to_num_tasks.put(key, val);
+  }
+
+  public Map<String,Long> get_component_to_num_tasks() {
+    return this.component_to_num_tasks;
+  }
+
+  public void set_component_to_num_tasks(Map<String,Long> component_to_num_tasks) {
+    this.component_to_num_tasks = component_to_num_tasks;
+  }
+
+  public void unset_component_to_num_tasks() {
+    this.component_to_num_tasks = null;
+  }
+
+  /** Returns true if field component_to_num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_to_num_tasks() {
+    return this.component_to_num_tasks != null;
+  }
+
+  public void set_component_to_num_tasks_isSet(boolean value) {
+    if (!value) {
+      this.component_to_num_tasks = null;
+    }
+  }
+
+  public int get_time_secs() {
+    return this.time_secs;
+  }
+
+  public void set_time_secs(int time_secs) {
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  public void unset_time_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  public void set_time_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SUPERVISOR_ID:
+      if (value == null) {
+        unset_supervisor_id();
+      } else {
+        set_supervisor_id((String)value);
+      }
+      break;
+
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topology_id();
+      } else {
+        set_topology_id((String)value);
+      }
+      break;
+
+    case TOPOLOGY_NAME:
+      if (value == null) {
+        unset_topology_name();
+      } else {
+        set_topology_name((String)value);
+      }
+      break;
+
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case COMPONENT_TO_NUM_TASKS:
+      if (value == null) {
+        unset_component_to_num_tasks();
+      } else {
+        set_component_to_num_tasks((Map<String,Long>)value);
+      }
+      break;
+
+    case TIME_SECS:
+      if (value == null) {
+        unset_time_secs();
+      } else {
+        set_time_secs((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SUPERVISOR_ID:
+      return get_supervisor_id();
+
+    case HOST:
+      return get_host();
+
+    case PORT:
+      return get_port();
+
+    case TOPOLOGY_ID:
+      return get_topology_id();
+
+    case TOPOLOGY_NAME:
+      return get_topology_name();
+
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case COMPONENT_TO_NUM_TASKS:
+      return get_component_to_num_tasks();
+
+    case TIME_SECS:
+      return get_time_secs();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case REQUESTED_MEMONHEAP:
+      return get_requested_memonheap();
+
+    case REQUESTED_MEMOFFHEAP:
+      return get_requested_memoffheap();
+
+    case REQUESTED_CPU:
+      return get_requested_cpu();
+
+    case ASSIGNED_MEMONHEAP:
+      return get_assigned_memonheap();
+
+    case ASSIGNED_MEMOFFHEAP:
+      return get_assigned_memoffheap();
+
+    case ASSIGNED_CPU:
+      return get_assigned_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SUPERVISOR_ID:
+      return is_set_supervisor_id();
+    case HOST:
+      return is_set_host();
+    case PORT:
+      return is_set_port();
+    case TOPOLOGY_ID:
+      return is_set_topology_id();
+    case TOPOLOGY_NAME:
+      return is_set_topology_name();
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case COMPONENT_TO_NUM_TASKS:
+      return is_set_component_to_num_tasks();
+    case TIME_SECS:
+      return is_set_time_secs();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WorkerSummary)
+      return this.equals((WorkerSummary)that);
+    return false;
+  }
+
+  public boolean equals(WorkerSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_supervisor_id = true && this.is_set_supervisor_id();
+    boolean that_present_supervisor_id = true && that.is_set_supervisor_id();
+    if (this_present_supervisor_id || that_present_supervisor_id) {
+      if (!(this_present_supervisor_id && that_present_supervisor_id))
+        return false;
+      if (!this.supervisor_id.equals(that.supervisor_id))
+        return false;
+    }
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_port = true && this.is_set_port();
+    boolean that_present_port = true && that.is_set_port();
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    boolean this_present_topology_id = true && this.is_set_topology_id();
+    boolean that_present_topology_id = true && that.is_set_topology_id();
+    if (this_present_topology_id || that_present_topology_id) {
+      if (!(this_present_topology_id && that_present_topology_id))
+        return false;
+      if (!this.topology_id.equals(that.topology_id))
+        return false;
+    }
+
+    boolean this_present_topology_name = true && this.is_set_topology_name();
+    boolean that_present_topology_name = true && that.is_set_topology_name();
+    if (this_present_topology_name || that_present_topology_name) {
+      if (!(this_present_topology_name && that_present_topology_name))
+        return false;
+      if (!this.topology_name.equals(that.topology_name))
+        return false;
+    }
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_component_to_num_tasks = true && this.is_set_component_to_num_tasks();
+    boolean that_present_component_to_num_tasks = true && that.is_set_component_to_num_tasks();
+    if (this_present_component_to_num_tasks || that_present_component_to_num_tasks) {
+      if (!(this_present_component_to_num_tasks && that_present_component_to_num_tasks))
+        return false;
+      if (!this.component_to_num_tasks.equals(that.component_to_num_tasks))
+        return false;
+    }
+
+    boolean this_present_time_secs = true && this.is_set_time_secs();
+    boolean that_present_time_secs = true && that.is_set_time_secs();
+    if (this_present_time_secs || that_present_time_secs) {
+      if (!(this_present_time_secs && that_present_time_secs))
+        return false;
+      if (this.time_secs != that.time_secs)
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true && this.is_set_uptime_secs();
+    boolean that_present_uptime_secs = true && that.is_set_uptime_secs();
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_supervisor_id = true && (is_set_supervisor_id());
+    list.add(present_supervisor_id);
+    if (present_supervisor_id)
+      list.add(supervisor_id);
+
+    boolean present_host = true && (is_set_host());
+    list.add(present_host);
+    if (present_host)
+      list.add(host);
+
+    boolean present_port = true && (is_set_port());
+    list.add(present_port);
+    if (present_port)
+      list.add(port);
+
+    boolean present_topology_id = true && (is_set_topology_id());
+    list.add(present_topology_id);
+    if (present_topology_id)
+      list.add(topology_id);
+
+    boolean present_topology_name = true && (is_set_topology_name());
+    list.add(present_topology_name);
+    if (present_topology_name)
+      list.add(topology_name);
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_component_to_num_tasks = true && (is_set_component_to_num_tasks());
+    list.add(present_component_to_num_tasks);
+    if (present_component_to_num_tasks)
+      list.add(component_to_num_tasks);
+
+    boolean present_time_secs = true && (is_set_time_secs());
+    list.add(present_time_secs);
+    if (present_time_secs)
+      list.add(time_secs);
+
+    boolean present_uptime_secs = true && (is_set_uptime_secs());
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WorkerSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(other.is_set_supervisor_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_supervisor_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_id, other.supervisor_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_name()).compareTo(other.is_set_topology_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_name, other.topology_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_to_num_tasks()).compareTo(other.is_set_component_to_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_to_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_to_num_tasks, other.component_to_num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WorkerSummary(");
+    boolean first = true;
+
+    if (is_set_supervisor_id()) {
+      sb.append("supervisor_id:");
+      if (this.supervisor_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.supervisor_id);
+      }
+      first = false;
+    }
+    if (is_set_host()) {
+      if (!first) sb.append(", ");
+      sb.append("host:");
+      if (this.host == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.host);
+      }
+      first = false;
+    }
+    if (is_set_port()) {
+      if (!first) sb.append(", ");
+      sb.append("port:");
+      sb.append(this.port);
+      first = false;
+    }
+    if (is_set_topology_id()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_id:");
+      if (this.topology_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_id);
+      }
+      first = false;
+    }
+    if (is_set_topology_name()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_name:");
+      if (this.topology_name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_name);
+      }
+      first = false;
+    }
+    if (is_set_num_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("num_executors:");
+      sb.append(this.num_executors);
+      first = false;
+    }
+    if (is_set_component_to_num_tasks()) {
+      if (!first) sb.append(", ");
+      sb.append("component_to_num_tasks:");
+      if (this.component_to_num_tasks == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_to_num_tasks);
+      }
+      first = false;
+    }
+    if (is_set_time_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("time_secs:");
+      sb.append(this.time_secs);
+      first = false;
+    }
+    if (is_set_uptime_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("uptime_secs:");
+      sb.append(this.uptime_secs);
+      first = false;
+    }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WorkerSummaryStandardSchemeFactory implements SchemeFactory {
+    public WorkerSummaryStandardScheme getScheme() {
+      return new WorkerSummaryStandardScheme();
+    }
+  }
+
+  private static class WorkerSummaryStandardScheme extends StandardScheme<WorkerSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SUPERVISOR_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.supervisor_id = iprot.readString();
+              struct.set_supervisor_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.host = iprot.readString();
+              struct.set_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.port = iprot.readI32();
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_id = iprot.readString();
+              struct.set_topology_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // TOPOLOGY_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_name = iprot.readString();
+              struct.set_topology_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // COMPONENT_TO_NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map428 = iprot.readMapBegin();
+                struct.component_to_num_tasks = new HashMap<String,Long>(2*_map428.size);
+                String _key429;
+                long _val430;
+                for (int _i431 = 0; _i431 < _map428.size; ++_i431)
+                {
+                  _key429 = iprot.readString();
+                  _val430 = iprot.readI64();
+                  struct.component_to_num_tasks.put(_key429, _val430);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_component_to_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.time_secs = iprot.readI32();
+              struct.set_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.supervisor_id != null) {
+        if (struct.is_set_supervisor_id()) {
+          oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC);
+          oprot.writeString(struct.supervisor_id);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.host != null) {
+        if (struct.is_set_host()) {
+          oprot.writeFieldBegin(HOST_FIELD_DESC);
+          oprot.writeString(struct.host);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_port()) {
+        oprot.writeFieldBegin(PORT_FIELD_DESC);
+        oprot.writeI32(struct.port);
+        oprot.writeFieldEnd();
+      }
+      if (struct.topology_id != null) {
+        if (struct.is_set_topology_id()) {
+          oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+          oprot.writeString(struct.topology_id);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_name != null) {
+        if (struct.is_set_topology_name()) {
+          oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC);
+          oprot.writeString(struct.topology_name);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+        oprot.writeI32(struct.num_executors);
+        oprot.writeFieldEnd();
+      }
+      if (struct.component_to_num_tasks != null) {
+        if (struct.is_set_component_to_num_tasks()) {
+          oprot.writeFieldBegin(COMPONENT_TO_NUM_TASKS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.component_to_num_tasks.size()));
+            for (Map.Entry<String, Long> _iter432 : struct.component_to_num_tasks.entrySet())
+            {
+              oprot.writeString(_iter432.getKey());
+              oprot.writeI64(_iter432.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_time_secs()) {
+        oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.time_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.uptime_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WorkerSummaryTupleSchemeFactory implements SchemeFactory {
+    public WorkerSummaryTupleScheme getScheme() {
+      return new WorkerSummaryTupleScheme();
+    }
+  }
+
+  private static class WorkerSummaryTupleScheme extends TupleScheme<WorkerSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WorkerSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_supervisor_id()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_host()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_port()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_topology_id()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_topology_name()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_num_executors()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_component_to_num_tasks()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_time_secs()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_uptime_secs()) {
+        optionals.set(8);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(9);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(10);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(11);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(12);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(13);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(14);
+      }
+      oprot.writeBitSet(optionals, 15);
+      if (struct.is_set_supervisor_id()) {
+        oprot.writeString(struct.supervisor_id);
+      }
+      if (struct.is_set_host()) {
+        oprot.writeString(struct.host);
+      }
+      if (struct.is_set_port()) {
+        oprot.writeI32(struct.port);
+      }
+      if (struct.is_set_topology_id()) {
+        oprot.writeString(struct.topology_id);
+      }
+      if (struct.is_set_topology_name()) {
+        oprot.writeString(struct.topology_name);
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeI32(struct.num_executors);
+      }
+      if (struct.is_set_component_to_num_tasks()) {
+        {
+          oprot.writeI32(struct.component_to_num_tasks.size());
+          for (Map.Entry<String, Long> _iter433 : struct.component_to_num_tasks.entrySet())
+          {
+            oprot.writeString(_iter433.getKey());
+            oprot.writeI64(_iter433.getValue());
+          }
+        }
+      }
+      if (struct.is_set_time_secs()) {
+        oprot.writeI32(struct.time_secs);
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeI32(struct.uptime_secs);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WorkerSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(15);
+      if (incoming.get(0)) {
+        struct.supervisor_id = iprot.readString();
+        struct.set_supervisor_id_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.host = iprot.readString();
+        struct.set_host_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.port = iprot.readI32();
+        struct.set_port_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.topology_id = iprot.readString();
+        struct.set_topology_id_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.topology_name = iprot.readString();
+        struct.set_topology_name_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.num_executors = iprot.readI32();
+        struct.set_num_executors_isSet(true);
+      }
+      if (incoming.get(6)) {
+        {
+          org.apache.thrift.protocol.TMap _map434 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.component_to_num_tasks = new HashMap<String,Long>(2*_map434.size);
+          String _key435;
+          long _val436;
+          for (int _i437 = 0; _i437 < _map434.size; ++_i437)
+          {
+            _key435 = iprot.readString();
+            _val436 = iprot.readI64();
+            struct.component_to_num_tasks.put(_key435, _val436);
+          }
+        }
+        struct.set_component_to_num_tasks_isSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.time_secs = iprot.readI32();
+        struct.set_time_secs_isSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.uptime_secs = iprot.readI32();
+        struct.set_uptime_secs_isSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(10)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(11)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(12)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(13)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(14)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java
new file mode 100644
index 0000000..8f72b82
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.grouping;
+
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.task.WorkerTopologyContext;
+import java.io.Serializable;
+import java.util.List;
+
+public interface CustomStreamGrouping extends Serializable {
+    
+   /**
+     * Tells the stream grouping at runtime the tasks in the target bolt.
+     * This information should be used in chooseTasks to determine the target tasks.
+     * 
+     * It also tells the grouping the metadata on the stream this grouping will be used on.
+     */
+   void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks);
+    
+   /**
+     * This function implements a custom stream grouping. It takes in as input
+     * the number of tasks in the target bolt in prepare and returns the
+     * tasks to send the tuples to.
+     * 
+     * @param values the values to group on
+     */
+   List<Integer> chooseTasks(int taskId, List<Object> values); 
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/grouping/Load.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/Load.java b/storm-client/src/jvm/org/apache/storm/grouping/Load.java
new file mode 100644
index 0000000..c0f5595
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/Load.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.grouping;
+
+import java.io.Serializable;
+
+/**
+ * Represents the load that a Bolt is currently under to help in
+ * deciding where to route a tuple, to help balance the load.
+ */
+public class Load {
+    private boolean hasMetrics = false;
+    private double boltLoad = 0.0; //0 no load to 1 fully loaded
+    private double connectionLoad = 0.0; //0 no load to 1 fully loaded
+
+    /**
+     * Create a new load
+     * @param hasMetrics have metrics been reported yet?
+     * @param boltLoad the load as reported by the bolt 0.0 no load 1.0 fully loaded
+     * @param connectionLoad the load as reported by the connection to the bolt 0.0 no load 1.0 fully loaded.
+     */
+    public Load(boolean hasMetrics, double boltLoad, double connectionLoad) {
+        this.hasMetrics = hasMetrics;
+        this.boltLoad = boltLoad;
+        this.connectionLoad = connectionLoad;
+    }
+
+    /**
+     * @return true if metrics have been reported so far.
+     */
+    public boolean hasMetrics() {
+        return hasMetrics;
+    }
+
+    /**
+     * @return the load as reported by the bolt.
+     */
+    public double getBoltLoad() {
+        return boltLoad;
+    }
+
+    /**
+     * @return the load as reported by the connection
+     */
+    public double getConnectionLoad() {
+        return connectionLoad;
+    }
+
+    /**
+     * @return the load that is a combination of sub loads.
+     */
+    public double getLoad() {
+        if (!hasMetrics) {
+            return 1.0;
+        }
+        return connectionLoad > boltLoad ? connectionLoad : boltLoad;
+    }
+
+    public String toString() {
+        return "[:load "+boltLoad+" "+connectionLoad+"]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java
new file mode 100644
index 0000000..d825d2e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.grouping;
+
+import java.util.List;
+
+public interface LoadAwareCustomStreamGrouping extends CustomStreamGrouping {
+   List<Integer> chooseTasks(int taskId, List<Object> values, LoadMapping load);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java
new file mode 100644
index 0000000..9a07194
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.grouping;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.task.WorkerTopologyContext;
+
+public class LoadAwareShuffleGrouping implements LoadAwareCustomStreamGrouping, Serializable {
+    private Random random;
+    private List<Integer>[] rets;
+    private int[] targets;
+    private int[] loads;
+    private int total;
+    private long lastUpdate = 0;
+
+    @Override
+    public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+        random = new Random();
+        rets = (List<Integer>[])new List<?>[targetTasks.size()];
+        targets = new int[targetTasks.size()];
+        for (int i = 0; i < targets.length; i++) {
+            rets[i] = Arrays.asList(targetTasks.get(i));
+            targets[i] = targetTasks.get(i);
+        }
+        loads = new int[targets.length];
+    }
+
+    @Override
+    public List<Integer> chooseTasks(int taskId, List<Object> values) {
+        throw new RuntimeException("NOT IMPLEMENTED");
+    }
+
+    @Override
+    public List<Integer> chooseTasks(int taskId, List<Object> values, LoadMapping load) {
+        if ((lastUpdate + 1000) < System.currentTimeMillis()) {
+            int local_total = 0;
+            for (int i = 0; i < targets.length; i++) {
+                int val = (int)(101 - (load.get(targets[i]) * 100));
+                loads[i] = val;
+                local_total += val;
+            }
+            total = local_total;
+            lastUpdate = System.currentTimeMillis();
+        }
+        int selected = random.nextInt(total);
+        int sum = 0;
+        for (int i = 0; i < targets.length; i++) {
+            sum += loads[i];
+            if (selected < sum) {
+                return rets[i];
+            }
+        }
+        return rets[rets.length-1];
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java b/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java
new file mode 100644
index 0000000..a6373c9
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.grouping;
+
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.Map;
+import java.util.HashMap;
+
+/**
+ * Holds a list of the current loads
+ */
+public class LoadMapping {
+    private static final Load NOT_CONNECTED = new Load(false, 1.0, 1.0);
+    private final AtomicReference<Map<Integer,Load>> _local = new AtomicReference<Map<Integer,Load>>(new HashMap<Integer,Load>());
+    private final AtomicReference<Map<Integer,Load>> _remote = new AtomicReference<Map<Integer,Load>>(new HashMap<Integer,Load>());
+
+    public void setLocal(Map<Integer, Double> local) {
+        Map<Integer, Load> newLocal = new HashMap<Integer, Load>();
+        if (local != null) {
+          for (Map.Entry<Integer, Double> entry: local.entrySet()) {
+            newLocal.put(entry.getKey(), new Load(true, entry.getValue(), 0.0));
+          }
+        }
+        _local.set(newLocal);
+    }
+
+    public void setRemote(Map<Integer, Load> remote) {
+        if (remote != null) {
+          _remote.set(new HashMap<Integer, Load>(remote));
+        } else {
+          _remote.set(new HashMap<Integer, Load>());
+        }
+    }
+
+    public Load getLoad(int task) {
+        Load ret = _local.get().get(task);
+        if (ret == null) {
+          ret = _remote.get().get(task);
+        }
+        if (ret == null) {
+          ret = NOT_CONNECTED;
+        }
+        return ret;
+    }
+
+    public double get(int task) {
+        return getLoad(task).getLoad();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java
new file mode 100644
index 0000000..e1af16d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.grouping;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.Fields;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+
+public class PartialKeyGrouping implements CustomStreamGrouping, Serializable {
+    private static final long serialVersionUID = -447379837314000353L;
+    private List<Integer> targetTasks;
+    private long[] targetTaskStats;
+    private HashFunction h1 = Hashing.murmur3_128(13);
+    private HashFunction h2 = Hashing.murmur3_128(17);
+    private Fields fields = null;
+    private Fields outFields = null;
+
+    public PartialKeyGrouping() {
+        //Empty
+    }
+
+    public PartialKeyGrouping(Fields fields) {
+        this.fields = fields;
+    }
+
+    @Override
+    public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+        this.targetTasks = targetTasks;
+        targetTaskStats = new long[this.targetTasks.size()];
+        if (this.fields != null) {
+            this.outFields = context.getComponentOutputFields(stream);
+        }
+    }
+
+    @Override
+    public List<Integer> chooseTasks(int taskId, List<Object> values) {
+        List<Integer> boltIds = new ArrayList<>(1);
+        if (values.size() > 0) {
+            byte[] raw;
+            if (fields != null) {
+                List<Object> selectedFields = outFields.select(fields, values);
+                ByteBuffer out = ByteBuffer.allocate(selectedFields.size() * 4);
+                for (Object o: selectedFields) {
+                    if (o instanceof List) {
+                        out.putInt(Arrays.deepHashCode(((List)o).toArray()));
+                    } else if (o instanceof Object[]) {
+                        out.putInt(Arrays.deepHashCode((Object[])o));
+                    } else if (o instanceof byte[]) {
+                        out.putInt(Arrays.hashCode((byte[]) o));
+                    } else if (o instanceof short[]) {
+                        out.putInt(Arrays.hashCode((short[]) o));
+                    } else if (o instanceof int[]) {
+                        out.putInt(Arrays.hashCode((int[]) o));
+                    } else if (o instanceof long[]) {
+                        out.putInt(Arrays.hashCode((long[]) o));
+                    } else if (o instanceof char[]) {
+                        out.putInt(Arrays.hashCode((char[]) o));
+                    } else if (o instanceof float[]) {
+                        out.putInt(Arrays.hashCode((float[]) o));
+                    } else if (o instanceof double[]) {
+                        out.putInt(Arrays.hashCode((double[]) o));
+                    } else if (o instanceof boolean[]) {
+                        out.putInt(Arrays.hashCode((boolean[]) o));
+                    } else if (o != null) {
+                        out.putInt(o.hashCode());
+                    } else {
+                      out.putInt(0);
+                    }
+                }
+                raw = out.array();
+            } else {
+                raw = values.get(0).toString().getBytes(); // assume key is the first field
+            }
+            int firstChoice = (int) (Math.abs(h1.hashBytes(raw).asLong()) % this.targetTasks.size());
+            int secondChoice = (int) (Math.abs(h2.hashBytes(raw).asLong()) % this.targetTasks.size());
+            int selected = targetTaskStats[firstChoice] > targetTaskStats[secondChoice] ? secondChoice : firstChoice;
+            boltIds.add(targetTasks.get(selected));
+            targetTaskStats[selected]++;
+        }
+        return boltIds;
+    }
+}


[10/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java
new file mode 100644
index 0000000..5e774e2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java
@@ -0,0 +1,461 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHistoryInfo, TopologyHistoryInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyHistoryInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyHistoryInfo");
+
+  private static final org.apache.thrift.protocol.TField TOPO_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("topo_ids", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologyHistoryInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologyHistoryInfoTupleSchemeFactory());
+  }
+
+  private List<String> topo_ids; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPO_IDS((short)1, "topo_ids");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPO_IDS
+          return TOPO_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPO_IDS, new org.apache.thrift.meta_data.FieldMetaData("topo_ids", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyHistoryInfo.class, metaDataMap);
+  }
+
+  public TopologyHistoryInfo() {
+  }
+
+  public TopologyHistoryInfo(
+    List<String> topo_ids)
+  {
+    this();
+    this.topo_ids = topo_ids;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyHistoryInfo(TopologyHistoryInfo other) {
+    if (other.is_set_topo_ids()) {
+      List<String> __this__topo_ids = new ArrayList<String>(other.topo_ids);
+      this.topo_ids = __this__topo_ids;
+    }
+  }
+
+  public TopologyHistoryInfo deepCopy() {
+    return new TopologyHistoryInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topo_ids = null;
+  }
+
+  public int get_topo_ids_size() {
+    return (this.topo_ids == null) ? 0 : this.topo_ids.size();
+  }
+
+  public java.util.Iterator<String> get_topo_ids_iterator() {
+    return (this.topo_ids == null) ? null : this.topo_ids.iterator();
+  }
+
+  public void add_to_topo_ids(String elem) {
+    if (this.topo_ids == null) {
+      this.topo_ids = new ArrayList<String>();
+    }
+    this.topo_ids.add(elem);
+  }
+
+  public List<String> get_topo_ids() {
+    return this.topo_ids;
+  }
+
+  public void set_topo_ids(List<String> topo_ids) {
+    this.topo_ids = topo_ids;
+  }
+
+  public void unset_topo_ids() {
+    this.topo_ids = null;
+  }
+
+  /** Returns true if field topo_ids is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topo_ids() {
+    return this.topo_ids != null;
+  }
+
+  public void set_topo_ids_isSet(boolean value) {
+    if (!value) {
+      this.topo_ids = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPO_IDS:
+      if (value == null) {
+        unset_topo_ids();
+      } else {
+        set_topo_ids((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPO_IDS:
+      return get_topo_ids();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPO_IDS:
+      return is_set_topo_ids();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyHistoryInfo)
+      return this.equals((TopologyHistoryInfo)that);
+    return false;
+  }
+
+  public boolean equals(TopologyHistoryInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topo_ids = true && this.is_set_topo_ids();
+    boolean that_present_topo_ids = true && that.is_set_topo_ids();
+    if (this_present_topo_ids || that_present_topo_ids) {
+      if (!(this_present_topo_ids && that_present_topo_ids))
+        return false;
+      if (!this.topo_ids.equals(that.topo_ids))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topo_ids = true && (is_set_topo_ids());
+    list.add(present_topo_ids);
+    if (present_topo_ids)
+      list.add(topo_ids);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologyHistoryInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topo_ids()).compareTo(other.is_set_topo_ids());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topo_ids()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topo_ids, other.topo_ids);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyHistoryInfo(");
+    boolean first = true;
+
+    sb.append("topo_ids:");
+    if (this.topo_ids == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topo_ids);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologyHistoryInfoStandardSchemeFactory implements SchemeFactory {
+    public TopologyHistoryInfoStandardScheme getScheme() {
+      return new TopologyHistoryInfoStandardScheme();
+    }
+  }
+
+  private static class TopologyHistoryInfoStandardScheme extends StandardScheme<TopologyHistoryInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPO_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list784 = iprot.readListBegin();
+                struct.topo_ids = new ArrayList<String>(_list784.size);
+                String _elem785;
+                for (int _i786 = 0; _i786 < _list784.size; ++_i786)
+                {
+                  _elem785 = iprot.readString();
+                  struct.topo_ids.add(_elem785);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_topo_ids_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topo_ids != null) {
+        oprot.writeFieldBegin(TOPO_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.topo_ids.size()));
+          for (String _iter787 : struct.topo_ids)
+          {
+            oprot.writeString(_iter787);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TopologyHistoryInfoTupleSchemeFactory implements SchemeFactory {
+    public TopologyHistoryInfoTupleScheme getScheme() {
+      return new TopologyHistoryInfoTupleScheme();
+    }
+  }
+
+  private static class TopologyHistoryInfoTupleScheme extends TupleScheme<TopologyHistoryInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_topo_ids()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_topo_ids()) {
+        {
+          oprot.writeI32(struct.topo_ids.size());
+          for (String _iter788 : struct.topo_ids)
+          {
+            oprot.writeString(_iter788);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list789 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.topo_ids = new ArrayList<String>(_list789.size);
+          String _elem790;
+          for (int _i791 = 0; _i791 < _list789.size; ++_i791)
+          {
+            _elem790 = iprot.readString();
+            struct.topo_ids.add(_elem790);
+          }
+        }
+        struct.set_topo_ids_isSet(true);
+      }
+    }
+  }
+
+}
+


[22/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java
new file mode 100644
index 0000000..792c185
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java
@@ -0,0 +1,805 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LSTopoHistory._Fields>, java.io.Serializable, Cloneable, Comparable<LSTopoHistory> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSTopoHistory");
+
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TIME_STAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("time_stamp", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField USERS_FIELD_DESC = new org.apache.thrift.protocol.TField("users", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField GROUPS_FIELD_DESC = new org.apache.thrift.protocol.TField("groups", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSTopoHistoryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSTopoHistoryTupleSchemeFactory());
+  }
+
+  private String topology_id; // required
+  private long time_stamp; // required
+  private List<String> users; // required
+  private List<String> groups; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPOLOGY_ID((short)1, "topology_id"),
+    TIME_STAMP((short)2, "time_stamp"),
+    USERS((short)3, "users"),
+    GROUPS((short)4, "groups");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 2: // TIME_STAMP
+          return TIME_STAMP;
+        case 3: // USERS
+          return USERS;
+        case 4: // GROUPS
+          return GROUPS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_STAMP_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TIME_STAMP, new org.apache.thrift.meta_data.FieldMetaData("time_stamp", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.USERS, new org.apache.thrift.meta_data.FieldMetaData("users", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.GROUPS, new org.apache.thrift.meta_data.FieldMetaData("groups", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSTopoHistory.class, metaDataMap);
+  }
+
+  public LSTopoHistory() {
+  }
+
+  public LSTopoHistory(
+    String topology_id,
+    long time_stamp,
+    List<String> users,
+    List<String> groups)
+  {
+    this();
+    this.topology_id = topology_id;
+    this.time_stamp = time_stamp;
+    set_time_stamp_isSet(true);
+    this.users = users;
+    this.groups = groups;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSTopoHistory(LSTopoHistory other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_topology_id()) {
+      this.topology_id = other.topology_id;
+    }
+    this.time_stamp = other.time_stamp;
+    if (other.is_set_users()) {
+      List<String> __this__users = new ArrayList<String>(other.users);
+      this.users = __this__users;
+    }
+    if (other.is_set_groups()) {
+      List<String> __this__groups = new ArrayList<String>(other.groups);
+      this.groups = __this__groups;
+    }
+  }
+
+  public LSTopoHistory deepCopy() {
+    return new LSTopoHistory(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topology_id = null;
+    set_time_stamp_isSet(false);
+    this.time_stamp = 0;
+    this.users = null;
+    this.groups = null;
+  }
+
+  public String get_topology_id() {
+    return this.topology_id;
+  }
+
+  public void set_topology_id(String topology_id) {
+    this.topology_id = topology_id;
+  }
+
+  public void unset_topology_id() {
+    this.topology_id = null;
+  }
+
+  /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_id() {
+    return this.topology_id != null;
+  }
+
+  public void set_topology_id_isSet(boolean value) {
+    if (!value) {
+      this.topology_id = null;
+    }
+  }
+
+  public long get_time_stamp() {
+    return this.time_stamp;
+  }
+
+  public void set_time_stamp(long time_stamp) {
+    this.time_stamp = time_stamp;
+    set_time_stamp_isSet(true);
+  }
+
+  public void unset_time_stamp() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  /** Returns true if field time_stamp is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_stamp() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  public void set_time_stamp_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_STAMP_ISSET_ID, value);
+  }
+
+  public int get_users_size() {
+    return (this.users == null) ? 0 : this.users.size();
+  }
+
+  public java.util.Iterator<String> get_users_iterator() {
+    return (this.users == null) ? null : this.users.iterator();
+  }
+
+  public void add_to_users(String elem) {
+    if (this.users == null) {
+      this.users = new ArrayList<String>();
+    }
+    this.users.add(elem);
+  }
+
+  public List<String> get_users() {
+    return this.users;
+  }
+
+  public void set_users(List<String> users) {
+    this.users = users;
+  }
+
+  public void unset_users() {
+    this.users = null;
+  }
+
+  /** Returns true if field users is set (has been assigned a value) and false otherwise */
+  public boolean is_set_users() {
+    return this.users != null;
+  }
+
+  public void set_users_isSet(boolean value) {
+    if (!value) {
+      this.users = null;
+    }
+  }
+
+  public int get_groups_size() {
+    return (this.groups == null) ? 0 : this.groups.size();
+  }
+
+  public java.util.Iterator<String> get_groups_iterator() {
+    return (this.groups == null) ? null : this.groups.iterator();
+  }
+
+  public void add_to_groups(String elem) {
+    if (this.groups == null) {
+      this.groups = new ArrayList<String>();
+    }
+    this.groups.add(elem);
+  }
+
+  public List<String> get_groups() {
+    return this.groups;
+  }
+
+  public void set_groups(List<String> groups) {
+    this.groups = groups;
+  }
+
+  public void unset_groups() {
+    this.groups = null;
+  }
+
+  /** Returns true if field groups is set (has been assigned a value) and false otherwise */
+  public boolean is_set_groups() {
+    return this.groups != null;
+  }
+
+  public void set_groups_isSet(boolean value) {
+    if (!value) {
+      this.groups = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topology_id();
+      } else {
+        set_topology_id((String)value);
+      }
+      break;
+
+    case TIME_STAMP:
+      if (value == null) {
+        unset_time_stamp();
+      } else {
+        set_time_stamp((Long)value);
+      }
+      break;
+
+    case USERS:
+      if (value == null) {
+        unset_users();
+      } else {
+        set_users((List<String>)value);
+      }
+      break;
+
+    case GROUPS:
+      if (value == null) {
+        unset_groups();
+      } else {
+        set_groups((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      return get_topology_id();
+
+    case TIME_STAMP:
+      return get_time_stamp();
+
+    case USERS:
+      return get_users();
+
+    case GROUPS:
+      return get_groups();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPOLOGY_ID:
+      return is_set_topology_id();
+    case TIME_STAMP:
+      return is_set_time_stamp();
+    case USERS:
+      return is_set_users();
+    case GROUPS:
+      return is_set_groups();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSTopoHistory)
+      return this.equals((LSTopoHistory)that);
+    return false;
+  }
+
+  public boolean equals(LSTopoHistory that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topology_id = true && this.is_set_topology_id();
+    boolean that_present_topology_id = true && that.is_set_topology_id();
+    if (this_present_topology_id || that_present_topology_id) {
+      if (!(this_present_topology_id && that_present_topology_id))
+        return false;
+      if (!this.topology_id.equals(that.topology_id))
+        return false;
+    }
+
+    boolean this_present_time_stamp = true;
+    boolean that_present_time_stamp = true;
+    if (this_present_time_stamp || that_present_time_stamp) {
+      if (!(this_present_time_stamp && that_present_time_stamp))
+        return false;
+      if (this.time_stamp != that.time_stamp)
+        return false;
+    }
+
+    boolean this_present_users = true && this.is_set_users();
+    boolean that_present_users = true && that.is_set_users();
+    if (this_present_users || that_present_users) {
+      if (!(this_present_users && that_present_users))
+        return false;
+      if (!this.users.equals(that.users))
+        return false;
+    }
+
+    boolean this_present_groups = true && this.is_set_groups();
+    boolean that_present_groups = true && that.is_set_groups();
+    if (this_present_groups || that_present_groups) {
+      if (!(this_present_groups && that_present_groups))
+        return false;
+      if (!this.groups.equals(that.groups))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topology_id = true && (is_set_topology_id());
+    list.add(present_topology_id);
+    if (present_topology_id)
+      list.add(topology_id);
+
+    boolean present_time_stamp = true;
+    list.add(present_time_stamp);
+    if (present_time_stamp)
+      list.add(time_stamp);
+
+    boolean present_users = true && (is_set_users());
+    list.add(present_users);
+    if (present_users)
+      list.add(users);
+
+    boolean present_groups = true && (is_set_groups());
+    list.add(present_groups);
+    if (present_groups)
+      list.add(groups);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSTopoHistory other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_stamp()).compareTo(other.is_set_time_stamp());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_stamp()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_stamp, other.time_stamp);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_users()).compareTo(other.is_set_users());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_users()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.users, other.users);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_groups()).compareTo(other.is_set_groups());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_groups()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.groups, other.groups);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSTopoHistory(");
+    boolean first = true;
+
+    sb.append("topology_id:");
+    if (this.topology_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topology_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("time_stamp:");
+    sb.append(this.time_stamp);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("users:");
+    if (this.users == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.users);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("groups:");
+    if (this.groups == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.groups);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_topology_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topology_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_time_stamp()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_stamp' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_users()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'users' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_groups()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'groups' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSTopoHistoryStandardSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryStandardScheme getScheme() {
+      return new LSTopoHistoryStandardScheme();
+    }
+  }
+
+  private static class LSTopoHistoryStandardScheme extends StandardScheme<LSTopoHistory> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_id = iprot.readString();
+              struct.set_topology_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TIME_STAMP
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.time_stamp = iprot.readI64();
+              struct.set_time_stamp_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // USERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list750 = iprot.readListBegin();
+                struct.users = new ArrayList<String>(_list750.size);
+                String _elem751;
+                for (int _i752 = 0; _i752 < _list750.size; ++_i752)
+                {
+                  _elem751 = iprot.readString();
+                  struct.users.add(_elem751);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_users_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // GROUPS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list753 = iprot.readListBegin();
+                struct.groups = new ArrayList<String>(_list753.size);
+                String _elem754;
+                for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+                {
+                  _elem754 = iprot.readString();
+                  struct.groups.add(_elem754);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_groups_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topology_id != null) {
+        oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+        oprot.writeString(struct.topology_id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TIME_STAMP_FIELD_DESC);
+      oprot.writeI64(struct.time_stamp);
+      oprot.writeFieldEnd();
+      if (struct.users != null) {
+        oprot.writeFieldBegin(USERS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.users.size()));
+          for (String _iter756 : struct.users)
+          {
+            oprot.writeString(_iter756);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.groups != null) {
+        oprot.writeFieldBegin(GROUPS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groups.size()));
+          for (String _iter757 : struct.groups)
+          {
+            oprot.writeString(_iter757);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSTopoHistoryTupleSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryTupleScheme getScheme() {
+      return new LSTopoHistoryTupleScheme();
+    }
+  }
+
+  private static class LSTopoHistoryTupleScheme extends TupleScheme<LSTopoHistory> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.topology_id);
+      oprot.writeI64(struct.time_stamp);
+      {
+        oprot.writeI32(struct.users.size());
+        for (String _iter758 : struct.users)
+        {
+          oprot.writeString(_iter758);
+        }
+      }
+      {
+        oprot.writeI32(struct.groups.size());
+        for (String _iter759 : struct.groups)
+        {
+          oprot.writeString(_iter759);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.topology_id = iprot.readString();
+      struct.set_topology_id_isSet(true);
+      struct.time_stamp = iprot.readI64();
+      struct.set_time_stamp_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list760 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.users = new ArrayList<String>(_list760.size);
+        String _elem761;
+        for (int _i762 = 0; _i762 < _list760.size; ++_i762)
+        {
+          _elem761 = iprot.readString();
+          struct.users.add(_elem761);
+        }
+      }
+      struct.set_users_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list763 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.groups = new ArrayList<String>(_list763.size);
+        String _elem764;
+        for (int _i765 = 0; _i765 < _list763.size; ++_i765)
+        {
+          _elem764 = iprot.readString();
+          struct.groups.add(_elem764);
+        }
+      }
+      struct.set_groups_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java
new file mode 100644
index 0000000..4b9b150
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java
@@ -0,0 +1,460 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryList, LSTopoHistoryList._Fields>, java.io.Serializable, Cloneable, Comparable<LSTopoHistoryList> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSTopoHistoryList");
+
+  private static final org.apache.thrift.protocol.TField TOPO_HISTORY_FIELD_DESC = new org.apache.thrift.protocol.TField("topo_history", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSTopoHistoryListStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSTopoHistoryListTupleSchemeFactory());
+  }
+
+  private List<LSTopoHistory> topo_history; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPO_HISTORY((short)1, "topo_history");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPO_HISTORY
+          return TOPO_HISTORY;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPO_HISTORY, new org.apache.thrift.meta_data.FieldMetaData("topo_history", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LSTopoHistory.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSTopoHistoryList.class, metaDataMap);
+  }
+
+  public LSTopoHistoryList() {
+  }
+
+  public LSTopoHistoryList(
+    List<LSTopoHistory> topo_history)
+  {
+    this();
+    this.topo_history = topo_history;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSTopoHistoryList(LSTopoHistoryList other) {
+    if (other.is_set_topo_history()) {
+      List<LSTopoHistory> __this__topo_history = new ArrayList<LSTopoHistory>(other.topo_history.size());
+      for (LSTopoHistory other_element : other.topo_history) {
+        __this__topo_history.add(new LSTopoHistory(other_element));
+      }
+      this.topo_history = __this__topo_history;
+    }
+  }
+
+  public LSTopoHistoryList deepCopy() {
+    return new LSTopoHistoryList(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topo_history = null;
+  }
+
+  public int get_topo_history_size() {
+    return (this.topo_history == null) ? 0 : this.topo_history.size();
+  }
+
+  public java.util.Iterator<LSTopoHistory> get_topo_history_iterator() {
+    return (this.topo_history == null) ? null : this.topo_history.iterator();
+  }
+
+  public void add_to_topo_history(LSTopoHistory elem) {
+    if (this.topo_history == null) {
+      this.topo_history = new ArrayList<LSTopoHistory>();
+    }
+    this.topo_history.add(elem);
+  }
+
+  public List<LSTopoHistory> get_topo_history() {
+    return this.topo_history;
+  }
+
+  public void set_topo_history(List<LSTopoHistory> topo_history) {
+    this.topo_history = topo_history;
+  }
+
+  public void unset_topo_history() {
+    this.topo_history = null;
+  }
+
+  /** Returns true if field topo_history is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topo_history() {
+    return this.topo_history != null;
+  }
+
+  public void set_topo_history_isSet(boolean value) {
+    if (!value) {
+      this.topo_history = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPO_HISTORY:
+      if (value == null) {
+        unset_topo_history();
+      } else {
+        set_topo_history((List<LSTopoHistory>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPO_HISTORY:
+      return get_topo_history();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPO_HISTORY:
+      return is_set_topo_history();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSTopoHistoryList)
+      return this.equals((LSTopoHistoryList)that);
+    return false;
+  }
+
+  public boolean equals(LSTopoHistoryList that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topo_history = true && this.is_set_topo_history();
+    boolean that_present_topo_history = true && that.is_set_topo_history();
+    if (this_present_topo_history || that_present_topo_history) {
+      if (!(this_present_topo_history && that_present_topo_history))
+        return false;
+      if (!this.topo_history.equals(that.topo_history))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topo_history = true && (is_set_topo_history());
+    list.add(present_topo_history);
+    if (present_topo_history)
+      list.add(topo_history);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSTopoHistoryList other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topo_history()).compareTo(other.is_set_topo_history());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topo_history()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topo_history, other.topo_history);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSTopoHistoryList(");
+    boolean first = true;
+
+    sb.append("topo_history:");
+    if (this.topo_history == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topo_history);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_topo_history()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topo_history' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSTopoHistoryListStandardSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryListStandardScheme getScheme() {
+      return new LSTopoHistoryListStandardScheme();
+    }
+  }
+
+  private static class LSTopoHistoryListStandardScheme extends StandardScheme<LSTopoHistoryList> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPO_HISTORY
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list766 = iprot.readListBegin();
+                struct.topo_history = new ArrayList<LSTopoHistory>(_list766.size);
+                LSTopoHistory _elem767;
+                for (int _i768 = 0; _i768 < _list766.size; ++_i768)
+                {
+                  _elem767 = new LSTopoHistory();
+                  _elem767.read(iprot);
+                  struct.topo_history.add(_elem767);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_topo_history_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topo_history != null) {
+        oprot.writeFieldBegin(TOPO_HISTORY_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topo_history.size()));
+          for (LSTopoHistory _iter769 : struct.topo_history)
+          {
+            _iter769.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSTopoHistoryListTupleSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryListTupleScheme getScheme() {
+      return new LSTopoHistoryListTupleScheme();
+    }
+  }
+
+  private static class LSTopoHistoryListTupleScheme extends TupleScheme<LSTopoHistoryList> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.topo_history.size());
+        for (LSTopoHistory _iter770 : struct.topo_history)
+        {
+          _iter770.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list771 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topo_history = new ArrayList<LSTopoHistory>(_list771.size);
+        LSTopoHistory _elem772;
+        for (int _i773 = 0; _i773 < _list771.size; ++_i773)
+        {
+          _elem772 = new LSTopoHistory();
+          _elem772.read(iprot);
+          struct.topo_history.add(_elem772);
+        }
+      }
+      struct.set_topo_history_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java b/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java
new file mode 100644
index 0000000..1c42bde
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java
@@ -0,0 +1,755 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartbeat, LSWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<LSWorkerHeartbeat> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSWorkerHeartbeat");
+
+  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSWorkerHeartbeatStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSWorkerHeartbeatTupleSchemeFactory());
+  }
+
+  private int time_secs; // required
+  private String topology_id; // required
+  private List<ExecutorInfo> executors; // required
+  private int port; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TIME_SECS((short)1, "time_secs"),
+    TOPOLOGY_ID((short)2, "topology_id"),
+    EXECUTORS((short)3, "executors"),
+    PORT((short)4, "port");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TIME_SECS
+          return TIME_SECS;
+        case 2: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 3: // EXECUTORS
+          return EXECUTORS;
+        case 4: // PORT
+          return PORT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_SECS_ISSET_ID = 0;
+  private static final int __PORT_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("executors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class))));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSWorkerHeartbeat.class, metaDataMap);
+  }
+
+  public LSWorkerHeartbeat() {
+  }
+
+  public LSWorkerHeartbeat(
+    int time_secs,
+    String topology_id,
+    List<ExecutorInfo> executors,
+    int port)
+  {
+    this();
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+    this.topology_id = topology_id;
+    this.executors = executors;
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSWorkerHeartbeat(LSWorkerHeartbeat other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.time_secs = other.time_secs;
+    if (other.is_set_topology_id()) {
+      this.topology_id = other.topology_id;
+    }
+    if (other.is_set_executors()) {
+      List<ExecutorInfo> __this__executors = new ArrayList<ExecutorInfo>(other.executors.size());
+      for (ExecutorInfo other_element : other.executors) {
+        __this__executors.add(new ExecutorInfo(other_element));
+      }
+      this.executors = __this__executors;
+    }
+    this.port = other.port;
+  }
+
+  public LSWorkerHeartbeat deepCopy() {
+    return new LSWorkerHeartbeat(this);
+  }
+
+  @Override
+  public void clear() {
+    set_time_secs_isSet(false);
+    this.time_secs = 0;
+    this.topology_id = null;
+    this.executors = null;
+    set_port_isSet(false);
+    this.port = 0;
+  }
+
+  public int get_time_secs() {
+    return this.time_secs;
+  }
+
+  public void set_time_secs(int time_secs) {
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  public void unset_time_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  public void set_time_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_topology_id() {
+    return this.topology_id;
+  }
+
+  public void set_topology_id(String topology_id) {
+    this.topology_id = topology_id;
+  }
+
+  public void unset_topology_id() {
+    this.topology_id = null;
+  }
+
+  /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_id() {
+    return this.topology_id != null;
+  }
+
+  public void set_topology_id_isSet(boolean value) {
+    if (!value) {
+      this.topology_id = null;
+    }
+  }
+
+  public int get_executors_size() {
+    return (this.executors == null) ? 0 : this.executors.size();
+  }
+
+  public java.util.Iterator<ExecutorInfo> get_executors_iterator() {
+    return (this.executors == null) ? null : this.executors.iterator();
+  }
+
+  public void add_to_executors(ExecutorInfo elem) {
+    if (this.executors == null) {
+      this.executors = new ArrayList<ExecutorInfo>();
+    }
+    this.executors.add(elem);
+  }
+
+  public List<ExecutorInfo> get_executors() {
+    return this.executors;
+  }
+
+  public void set_executors(List<ExecutorInfo> executors) {
+    this.executors = executors;
+  }
+
+  public void unset_executors() {
+    this.executors = null;
+  }
+
+  /** Returns true if field executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executors() {
+    return this.executors != null;
+  }
+
+  public void set_executors_isSet(boolean value) {
+    if (!value) {
+      this.executors = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TIME_SECS:
+      if (value == null) {
+        unset_time_secs();
+      } else {
+        set_time_secs((Integer)value);
+      }
+      break;
+
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topology_id();
+      } else {
+        set_topology_id((String)value);
+      }
+      break;
+
+    case EXECUTORS:
+      if (value == null) {
+        unset_executors();
+      } else {
+        set_executors((List<ExecutorInfo>)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TIME_SECS:
+      return get_time_secs();
+
+    case TOPOLOGY_ID:
+      return get_topology_id();
+
+    case EXECUTORS:
+      return get_executors();
+
+    case PORT:
+      return get_port();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TIME_SECS:
+      return is_set_time_secs();
+    case TOPOLOGY_ID:
+      return is_set_topology_id();
+    case EXECUTORS:
+      return is_set_executors();
+    case PORT:
+      return is_set_port();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSWorkerHeartbeat)
+      return this.equals((LSWorkerHeartbeat)that);
+    return false;
+  }
+
+  public boolean equals(LSWorkerHeartbeat that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_time_secs = true;
+    boolean that_present_time_secs = true;
+    if (this_present_time_secs || that_present_time_secs) {
+      if (!(this_present_time_secs && that_present_time_secs))
+        return false;
+      if (this.time_secs != that.time_secs)
+        return false;
+    }
+
+    boolean this_present_topology_id = true && this.is_set_topology_id();
+    boolean that_present_topology_id = true && that.is_set_topology_id();
+    if (this_present_topology_id || that_present_topology_id) {
+      if (!(this_present_topology_id && that_present_topology_id))
+        return false;
+      if (!this.topology_id.equals(that.topology_id))
+        return false;
+    }
+
+    boolean this_present_executors = true && this.is_set_executors();
+    boolean that_present_executors = true && that.is_set_executors();
+    if (this_present_executors || that_present_executors) {
+      if (!(this_present_executors && that_present_executors))
+        return false;
+      if (!this.executors.equals(that.executors))
+        return false;
+    }
+
+    boolean this_present_port = true;
+    boolean that_present_port = true;
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_time_secs = true;
+    list.add(present_time_secs);
+    if (present_time_secs)
+      list.add(time_secs);
+
+    boolean present_topology_id = true && (is_set_topology_id());
+    list.add(present_topology_id);
+    if (present_topology_id)
+      list.add(topology_id);
+
+    boolean present_executors = true && (is_set_executors());
+    list.add(present_executors);
+    if (present_executors)
+      list.add(executors);
+
+    boolean present_port = true;
+    list.add(present_port);
+    if (present_port)
+      list.add(port);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSWorkerHeartbeat other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executors()).compareTo(other.is_set_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executors, other.executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSWorkerHeartbeat(");
+    boolean first = true;
+
+    sb.append("time_secs:");
+    sb.append(this.time_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("topology_id:");
+    if (this.topology_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topology_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executors:");
+    if (this.executors == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executors);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    sb.append(this.port);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_time_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_topology_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topology_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executors()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executors' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSWorkerHeartbeatStandardSchemeFactory implements SchemeFactory {
+    public LSWorkerHeartbeatStandardScheme getScheme() {
+      return new LSWorkerHeartbeatStandardScheme();
+    }
+  }
+
+  private static class LSWorkerHeartbeatStandardScheme extends StandardScheme<LSWorkerHeartbeat> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.time_secs = iprot.readI32();
+              struct.set_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_id = iprot.readString();
+              struct.set_topology_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list742 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list742.size);
+                ExecutorInfo _elem743;
+                for (int _i744 = 0; _i744 < _list742.size; ++_i744)
+                {
+                  _elem743 = new ExecutorInfo();
+                  _elem743.read(iprot);
+                  struct.executors.add(_elem743);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.port = iprot.readI32();
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.time_secs);
+      oprot.writeFieldEnd();
+      if (struct.topology_id != null) {
+        oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+        oprot.writeString(struct.topology_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.executors != null) {
+        oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
+          for (ExecutorInfo _iter745 : struct.executors)
+          {
+            _iter745.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(PORT_FIELD_DESC);
+      oprot.writeI32(struct.port);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSWorkerHeartbeatTupleSchemeFactory implements SchemeFactory {
+    public LSWorkerHeartbeatTupleScheme getScheme() {
+      return new LSWorkerHeartbeatTupleScheme();
+    }
+  }
+
+  private static class LSWorkerHeartbeatTupleScheme extends TupleScheme<LSWorkerHeartbeat> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.time_secs);
+      oprot.writeString(struct.topology_id);
+      {
+        oprot.writeI32(struct.executors.size());
+        for (ExecutorInfo _iter746 : struct.executors)
+        {
+          _iter746.write(oprot);
+        }
+      }
+      oprot.writeI32(struct.port);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.time_secs = iprot.readI32();
+      struct.set_time_secs_isSet(true);
+      struct.topology_id = iprot.readString();
+      struct.set_topology_id_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list747 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list747.size);
+        ExecutorInfo _elem748;
+        for (int _i749 = 0; _i749 < _list747.size; ++_i749)
+        {
+          _elem748 = new ExecutorInfo();
+          _elem748.read(iprot);
+          struct.executors.add(_elem748);
+        }
+      }
+      struct.set_executors_isSet(true);
+      struct.port = iprot.readI32();
+      struct.set_port_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java b/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java
new file mode 100644
index 0000000..87337e1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java
@@ -0,0 +1,556 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ListBlobsResult implements org.apache.thrift.TBase<ListBlobsResult, ListBlobsResult._Fields>, java.io.Serializable, Cloneable, Comparable<ListBlobsResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ListBlobsResult");
+
+  private static final org.apache.thrift.protocol.TField KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("keys", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField SESSION_FIELD_DESC = new org.apache.thrift.protocol.TField("session", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ListBlobsResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ListBlobsResultTupleSchemeFactory());
+  }
+
+  private List<String> keys; // required
+  private String session; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    KEYS((short)1, "keys"),
+    SESSION((short)2, "session");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // KEYS
+          return KEYS;
+        case 2: // SESSION
+          return SESSION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.SESSION, new org.apache.thrift.meta_data.FieldMetaData("session", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ListBlobsResult.class, metaDataMap);
+  }
+
+  public ListBlobsResult() {
+  }
+
+  public ListBlobsResult(
+    List<String> keys,
+    String session)
+  {
+    this();
+    this.keys = keys;
+    this.session = session;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ListBlobsResult(ListBlobsResult other) {
+    if (other.is_set_keys()) {
+      List<String> __this__keys = new ArrayList<String>(other.keys);
+      this.keys = __this__keys;
+    }
+    if (other.is_set_session()) {
+      this.session = other.session;
+    }
+  }
+
+  public ListBlobsResult deepCopy() {
+    return new ListBlobsResult(this);
+  }
+
+  @Override
+  public void clear() {
+    this.keys = null;
+    this.session = null;
+  }
+
+  public int get_keys_size() {
+    return (this.keys == null) ? 0 : this.keys.size();
+  }
+
+  public java.util.Iterator<String> get_keys_iterator() {
+    return (this.keys == null) ? null : this.keys.iterator();
+  }
+
+  public void add_to_keys(String elem) {
+    if (this.keys == null) {
+      this.keys = new ArrayList<String>();
+    }
+    this.keys.add(elem);
+  }
+
+  public List<String> get_keys() {
+    return this.keys;
+  }
+
+  public void set_keys(List<String> keys) {
+    this.keys = keys;
+  }
+
+  public void unset_keys() {
+    this.keys = null;
+  }
+
+  /** Returns true if field keys is set (has been assigned a value) and false otherwise */
+  public boolean is_set_keys() {
+    return this.keys != null;
+  }
+
+  public void set_keys_isSet(boolean value) {
+    if (!value) {
+      this.keys = null;
+    }
+  }
+
+  public String get_session() {
+    return this.session;
+  }
+
+  public void set_session(String session) {
+    this.session = session;
+  }
+
+  public void unset_session() {
+    this.session = null;
+  }
+
+  /** Returns true if field session is set (has been assigned a value) and false otherwise */
+  public boolean is_set_session() {
+    return this.session != null;
+  }
+
+  public void set_session_isSet(boolean value) {
+    if (!value) {
+      this.session = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case KEYS:
+      if (value == null) {
+        unset_keys();
+      } else {
+        set_keys((List<String>)value);
+      }
+      break;
+
+    case SESSION:
+      if (value == null) {
+        unset_session();
+      } else {
+        set_session((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case KEYS:
+      return get_keys();
+
+    case SESSION:
+      return get_session();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case KEYS:
+      return is_set_keys();
+    case SESSION:
+      return is_set_session();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ListBlobsResult)
+      return this.equals((ListBlobsResult)that);
+    return false;
+  }
+
+  public boolean equals(ListBlobsResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_keys = true && this.is_set_keys();
+    boolean that_present_keys = true && that.is_set_keys();
+    if (this_present_keys || that_present_keys) {
+      if (!(this_present_keys && that_present_keys))
+        return false;
+      if (!this.keys.equals(that.keys))
+        return false;
+    }
+
+    boolean this_present_session = true && this.is_set_session();
+    boolean that_present_session = true && that.is_set_session();
+    if (this_present_session || that_present_session) {
+      if (!(this_present_session && that_present_session))
+        return false;
+      if (!this.session.equals(that.session))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_keys = true && (is_set_keys());
+    list.add(present_keys);
+    if (present_keys)
+      list.add(keys);
+
+    boolean present_session = true && (is_set_session());
+    list.add(present_session);
+    if (present_session)
+      list.add(session);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ListBlobsResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_keys()).compareTo(other.is_set_keys());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_keys()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keys, other.keys);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_session()).compareTo(other.is_set_session());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_session()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.session, other.session);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ListBlobsResult(");
+    boolean first = true;
+
+    sb.append("keys:");
+    if (this.keys == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.keys);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("session:");
+    if (this.session == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.session);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_keys()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'keys' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_session()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'session' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ListBlobsResultStandardSchemeFactory implements SchemeFactory {
+    public ListBlobsResultStandardScheme getScheme() {
+      return new ListBlobsResultStandardScheme();
+    }
+  }
+
+  private static class ListBlobsResultStandardScheme extends StandardScheme<ListBlobsResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // KEYS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
+                struct.keys = new ArrayList<String>(_list566.size);
+                String _elem567;
+                for (int _i568 = 0; _i568 < _list566.size; ++_i568)
+                {
+                  _elem567 = iprot.readString();
+                  struct.keys.add(_elem567);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_keys_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SESSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.session = iprot.readString();
+              struct.set_session_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.keys != null) {
+        oprot.writeFieldBegin(KEYS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.keys.size()));
+          for (String _iter569 : struct.keys)
+          {
+            oprot.writeString(_iter569);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.session != null) {
+        oprot.writeFieldBegin(SESSION_FIELD_DESC);
+        oprot.writeString(struct.session);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ListBlobsResultTupleSchemeFactory implements SchemeFactory {
+    public ListBlobsResultTupleScheme getScheme() {
+      return new ListBlobsResultTupleScheme();
+    }
+  }
+
+  private static class ListBlobsResultTupleScheme extends TupleScheme<ListBlobsResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.keys.size());
+        for (String _iter570 : struct.keys)
+        {
+          oprot.writeString(_iter570);
+        }
+      }
+      oprot.writeString(struct.session);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.keys = new ArrayList<String>(_list571.size);
+        String _elem572;
+        for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+        {
+          _elem572 = iprot.readString();
+          struct.keys.add(_elem572);
+        }
+      }
+      struct.set_keys_isSet(true);
+      struct.session = iprot.readString();
+      struct.set_session_isSet(true);
+    }
+  }
+
+}
+


[40/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/worker/LogConfigManager.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/LogConfigManager.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/LogConfigManager.java
new file mode 100644
index 0000000..00fccf9
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/LogConfigManager.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.daemon.worker;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.storm.utils.Time;
+import org.apache.storm.generated.LogConfig;
+import org.apache.storm.generated.LogLevel;
+import org.apache.storm.generated.LogLevelAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class LogConfigManager {
+
+    private static final Logger LOG = LoggerFactory.getLogger(LogConfigManager.class);
+
+    private final AtomicReference<TreeMap<String, LogLevel>> latestLogConfig;
+    private final Map<String, Level> originalLogLevels;
+
+    public LogConfigManager() {
+        this(new AtomicReference<>(new TreeMap<>()));
+    }
+
+    public LogConfigManager(AtomicReference<TreeMap<String, LogLevel>> latestLogConfig) {
+        this.latestLogConfig = latestLogConfig;
+        this.originalLogLevels = getLoggerLevels();
+        LOG.info("Started with log levels: {}", originalLogLevels);
+    }
+
+    public void processLogConfigChange(LogConfig logConfig) {
+        if (null != logConfig) {
+            LOG.debug("Processing received log config: {}", logConfig);
+            TreeMap<String, LogLevel> loggers = new TreeMap<>(logConfig.get_named_logger_level());
+            LoggerContext logContext = (LoggerContext) LogManager.getContext(false);
+            Map<String, LogLevel> newLogConfigs = new HashMap<>();
+            for (Map.Entry<String, LogLevel> entry : loggers.entrySet()) {
+                String msgLoggerName = entry.getKey();
+                msgLoggerName = ("ROOT".equalsIgnoreCase(msgLoggerName)) ? LogManager.ROOT_LOGGER_NAME : msgLoggerName;
+                LogLevel loggerLevel = entry.getValue();
+                // the new-timeouts map now contains logger => timeout
+                if (loggerLevel.is_set_reset_log_level_timeout_epoch()) {
+                    LogLevel copy = new LogLevel(loggerLevel);
+                    if (originalLogLevels.containsKey(msgLoggerName)) {
+                        copy.set_reset_log_level(originalLogLevels.get(msgLoggerName).name());
+                    } else {
+                        copy.set_reset_log_level(Level.INFO.name());
+                    }
+
+                    newLogConfigs.put(msgLoggerName, copy);
+                }
+
+            }
+
+            // Look for deleted log timeouts
+            TreeMap<String,LogLevel> latestConf = latestLogConfig.get();
+            if (latestConf != null) {
+                for (String loggerName : latestConf.descendingKeySet()) {
+                    if (! newLogConfigs.containsKey(loggerName)) {
+                        // if we had a timeout, but the timeout is no longer active
+                        setLoggerLevel(logContext, loggerName, latestConf.get(loggerName).get_reset_log_level());
+
+                    }
+                }
+            }
+
+            // apply new log settings we just received
+            // the merged configs are only for the reset logic
+            for (String loggerName : new TreeSet<>(logConfig.get_named_logger_level().keySet())) {
+                LogLevel logLevel = logConfig.get_named_logger_level().get(loggerName);
+                loggerName = ("ROOT".equalsIgnoreCase(loggerName)) ? LogManager.ROOT_LOGGER_NAME : loggerName;
+                LogLevelAction action = logLevel.get_action();
+                if (action == LogLevelAction.UPDATE) {
+                    setLoggerLevel(logContext, loggerName, logLevel.get_target_log_level());
+                }
+
+            }
+
+            logContext.updateLoggers();
+            latestLogConfig.set(new TreeMap<>(newLogConfigs));
+            LOG.debug("New merged log config is {}", latestLogConfig.get());
+        }
+    }
+
+    // function called on timer to reset log levels last set to DEBUG
+    // also called from processLogConfigChange
+    public void resetLogLevels() {
+        TreeMap<String, LogLevel> latestLogLevelMap = latestLogConfig.get();
+        LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
+
+        for (String loggerName : latestLogLevelMap.descendingKeySet()) {
+            LogLevel loggerSetting = latestLogLevelMap.get(loggerName);
+            long timeout = loggerSetting.get_reset_log_level_timeout_epoch();
+            String resetLogLevel = loggerSetting.get_reset_log_level();
+            if (timeout < Time.currentTimeMillis()) {
+                LOG.info("{}: Resetting level to {}", loggerName, resetLogLevel);
+                setLoggerLevel(loggerContext, loggerName, resetLogLevel);
+            }
+            latestLogConfig.getAndUpdate(input -> {
+                TreeMap<String, LogLevel> result = new TreeMap<>(input);
+                result.remove(loggerName);
+                return result;
+            });
+        }
+        loggerContext.updateLoggers();
+    }
+
+    public Map<String, Level> getLoggerLevels() {
+        Configuration loggerConfig = ((LoggerContext) LogManager.getContext(false)).getConfiguration();
+        Map<String, Level> logLevelMap = new HashMap<>();
+        for (Map.Entry<String, LoggerConfig> entry : loggerConfig.getLoggers().entrySet()) {
+            logLevelMap.put(entry.getKey(), entry.getValue().getLevel());
+        }
+        return logLevelMap;
+    }
+
+    public void setLoggerLevel(LoggerContext logContext, String loggerName, String newLevelStr) {
+        Level newLevel = Level.getLevel(newLevelStr);
+        Configuration configuration = logContext.getConfiguration();
+        LoggerConfig loggerConfig = configuration.getLoggerConfig(loggerName);
+        if (loggerConfig.getName().equalsIgnoreCase(loggerName)) {
+            LOG.info("Setting {} log level to: {}", loggerConfig, newLevel);
+            loggerConfig.setLevel(newLevel);
+        } else {
+            // create a new config. Make it additive (true) s.t. inherit parents appenders
+            LoggerConfig newLoggerConfig = new LoggerConfig(loggerName, newLevel, true);
+            LOG.info("Adding config for: {} with level: {}", newLoggerConfig, newLevel);
+            configuration.addLogger(loggerName, newLoggerConfig);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
new file mode 100644
index 0000000..74f5c5f
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
@@ -0,0 +1,459 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.daemon.worker;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.security.auth.Subject;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStateStorage;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.Credentials;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.LSWorkerHeartbeat;
+import org.apache.storm.generated.LogConfig;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.LocalState;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.WorkerBackpressureCallback;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Worker.class);
+    private final Map conf;
+    private final IContext context;
+    private final String topologyId;
+    private final String assignmentId;
+    private final int port;
+    private final String workerId;
+    private final LogConfigManager logConfigManager;
+
+
+    private WorkerState workerState;
+    private AtomicReference<List<IRunningExecutor>> executorsAtom;
+    private Thread transferThread;
+    private WorkerBackpressureThread backpressureThread;
+
+    private AtomicReference<Credentials> credentialsAtom;
+    private Subject subject;
+    private Collection<IAutoCredentials> autoCreds;
+
+
+    /**
+     * TODO: should worker even take the topologyId as input? this should be
+     * deducable from cluster state (by searching through assignments)
+     * what about if there's inconsistency in assignments? -> but nimbus should guarantee this consistency
+     *
+     * @param conf         - Storm configuration
+     * @param context      -
+     * @param topologyId   - topology id
+     * @param assignmentId - assignement id
+     * @param port         - port on which the worker runs
+     * @param workerId     - worker id
+     */
+
+    public Worker(Map conf, IContext context, String topologyId, String assignmentId, int port, String workerId) {
+        this.conf = conf;
+        this.context = context;
+        this.topologyId = topologyId;
+        this.assignmentId = assignmentId;
+        this.port = port;
+        this.workerId = workerId;
+        this.logConfigManager = new LogConfigManager();
+    }
+
+    public void start() throws Exception {
+        LOG.info("Launching worker for {} on {}:{} with id {} and conf {}", topologyId, assignmentId, port, workerId,
+            conf);
+        // because in local mode, its not a separate
+        // process. supervisor will register it in this case
+        // if ConfigUtils.isLocalMode(conf) returns false then it is in distributed mode.
+        if (!ConfigUtils.isLocalMode(conf)) {
+            // Distributed mode
+            SysOutOverSLF4J.sendSystemOutAndErrToSLF4J();
+            String pid = Utils.processPid();
+            FileUtils.touch(new File(ConfigUtils.workerPidPath(conf, workerId, pid)));
+            FileUtils.writeStringToFile(new File(ConfigUtils.workerArtifactsPidPath(conf, topologyId, port)), pid,
+                Charset.forName("UTF-8"));
+        }
+        final Map topologyConf =
+            ConfigUtils.overrideLoginConfigWithSystemProperty(ConfigUtils.readSupervisorStormConf(conf, topologyId));
+        List<ACL> acls = Utils.getWorkerACL(topologyConf);
+        IStateStorage stateStorage =
+            ClusterUtils.mkStateStorage(conf, topologyConf, acls, new ClusterStateContext(DaemonType.WORKER));
+        IStormClusterState stormClusterState =
+            ClusterUtils.mkStormClusterState(stateStorage, acls, new ClusterStateContext());
+        Credentials initialCredentials = stormClusterState.credentials(topologyId, null);
+        Map<String, String> initCreds = new HashMap<>();
+        if (initialCredentials != null) {
+            initCreds.putAll(initialCredentials.get_creds());
+        }
+        autoCreds = AuthUtils.GetAutoCredentials(topologyConf);
+        subject = AuthUtils.populateSubject(null, autoCreds, initCreds);
+
+        Subject.doAs(subject, new PrivilegedExceptionAction<Object>() {
+            @Override public Object run() throws Exception {
+                workerState =
+                    new WorkerState(conf, context, topologyId, assignmentId, port, workerId, topologyConf, stateStorage,
+                        stormClusterState);
+
+                // Heartbeat here so that worker process dies if this fails
+                // it's important that worker heartbeat to supervisor ASAP so that supervisor knows
+                // that worker is running and moves on
+                doHeartBeat();
+
+                executorsAtom = new AtomicReference<>(null);
+
+                // launch heartbeat threads immediately so that slow-loading tasks don't cause the worker to timeout
+                // to the supervisor
+                workerState.heartbeatTimer
+                    .scheduleRecurring(0, (Integer) conf.get(Config.WORKER_HEARTBEAT_FREQUENCY_SECS), () -> {
+                        try {
+                            doHeartBeat();
+                        } catch (IOException e) {
+                            throw new RuntimeException(e);
+                        }
+                    });
+
+                workerState.executorHeartbeatTimer
+                    .scheduleRecurring(0, (Integer) conf.get(Config.WORKER_HEARTBEAT_FREQUENCY_SECS),
+                        Worker.this::doExecutorHeartbeats);
+
+                workerState.registerCallbacks();
+
+                workerState.refreshConnections(null);
+
+                workerState.activateWorkerWhenAllConnectionsReady();
+
+                workerState.refreshStormActive(null);
+
+                workerState.runWorkerStartHooks();
+
+                List<IRunningExecutor> newExecutors = new ArrayList<IRunningExecutor>();
+                for (List<Long> e : workerState.getExecutors()) {
+                    if (ConfigUtils.isLocalMode(topologyConf)) {
+                        newExecutors.add(
+                            LocalExecutor.mkExecutor(workerState, e, initCreds)
+                                .execute());
+                    } else {
+                        newExecutors.add(
+                            Executor.mkExecutor(workerState, e, initCreds)
+                                .execute());
+                    }
+                }
+                executorsAtom.set(newExecutors);
+
+                EventHandler<Object> tupleHandler = (packets, seqId, batchEnd) -> workerState
+                    .sendTuplesToRemoteWorker((HashMap<Integer, ArrayList<TaskMessage>>) packets, seqId, batchEnd);
+
+                // This thread will publish the messages destined for remote tasks to remote connections
+                transferThread = Utils.asyncLoop(() -> {
+                    workerState.transferQueue.consumeBatchWhenAvailable(tupleHandler);
+                    return 0L;
+                });
+
+                DisruptorBackpressureCallback disruptorBackpressureHandler =
+                    mkDisruptorBackpressureHandler(workerState);
+                workerState.transferQueue.registerBackpressureCallback(disruptorBackpressureHandler);
+                workerState.transferQueue
+                    .setEnableBackpressure((Boolean) topologyConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE));
+                workerState.transferQueue
+                    .setHighWaterMark(ObjectReader.getDouble(topologyConf.get(Config.BACKPRESSURE_DISRUPTOR_HIGH_WATERMARK)));
+                workerState.transferQueue
+                    .setLowWaterMark(ObjectReader.getDouble(topologyConf.get(Config.BACKPRESSURE_DISRUPTOR_LOW_WATERMARK)));
+
+                WorkerBackpressureCallback backpressureCallback = mkBackpressureHandler();
+                backpressureThread = new WorkerBackpressureThread(workerState.backpressureTrigger, workerState, backpressureCallback);
+                if ((Boolean) topologyConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE)) {
+                    backpressureThread.start();
+                    stormClusterState.topologyBackpressure(topologyId, workerState::refreshThrottle);
+                    
+                    int pollingSecs = ObjectReader.getInt(topologyConf.get(Config.TASK_BACKPRESSURE_POLL_SECS));
+                    workerState.refreshBackpressureTimer.scheduleRecurring(0, pollingSecs, workerState::refreshThrottle);
+                }
+
+                credentialsAtom = new AtomicReference<Credentials>(initialCredentials);
+
+                establishLogSettingCallback();
+
+                workerState.stormClusterState.credentials(topologyId, Worker.this::checkCredentialsChanged);
+
+                workerState.refreshCredentialsTimer.scheduleRecurring(0,
+                    (Integer) conf.get(Config.TASK_CREDENTIALS_POLL_SECS), new Runnable() {
+                        @Override public void run() {
+                            checkCredentialsChanged();
+                            if ((Boolean) topologyConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE)) {
+                               checkThrottleChanged();
+                            }
+                        }
+                    });
+              
+                // The jitter allows the clients to get the data at different times, and avoids thundering herd
+                if (!(Boolean) topologyConf.get(Config.TOPOLOGY_DISABLE_LOADAWARE_MESSAGING)) {
+                    workerState.refreshLoadTimer.scheduleRecurringWithJitter(0, 1, 500, workerState::refreshLoad);
+                }
+
+                workerState.refreshConnectionsTimer.scheduleRecurring(0,
+                    (Integer) conf.get(Config.TASK_REFRESH_POLL_SECS), workerState::refreshConnections);
+
+                workerState.resetLogLevelsTimer.scheduleRecurring(0,
+                    (Integer) conf.get(Config.WORKER_LOG_LEVEL_RESET_POLL_SECS), logConfigManager::resetLogLevels);
+
+                workerState.refreshActiveTimer.scheduleRecurring(0, (Integer) conf.get(Config.TASK_REFRESH_POLL_SECS),
+                    workerState::refreshStormActive);
+
+                LOG.info("Worker has topology config {}", Utils.redactValue(topologyConf, Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD));
+                LOG.info("Worker {} for storm {} on {}:{}  has finished loading", workerId, topologyId, assignmentId, port);
+                return this;
+            };
+        });
+
+    }
+
+    public void doHeartBeat() throws IOException {
+        LocalState state = ConfigUtils.workerState(workerState.conf, workerState.workerId);
+        state.setWorkerHeartBeat(new LSWorkerHeartbeat(Time.currentTimeSecs(), workerState.topologyId,
+            workerState.executors.stream()
+                .map(executor -> new ExecutorInfo(executor.get(0).intValue(), executor.get(1).intValue()))
+                .collect(Collectors.toList()), workerState.port));
+        state.cleanup(60); // this is just in case supervisor is down so that disk doesn't fill up.
+        // it shouldn't take supervisor 120 seconds between listing dir and reading it
+    }
+
+    public void doExecutorHeartbeats() {
+        Map<List<Integer>, ExecutorStats> stats;
+        List<IRunningExecutor> executors = this.executorsAtom.get();
+        if (null == executors) {
+            stats = StatsUtil.mkEmptyExecutorZkHbs(workerState.executors);
+        } else {
+            stats = StatsUtil.convertExecutorZkHbs(executors.stream().collect(Collectors
+                .toMap((Function<IRunningExecutor, List<Long>>) IRunningExecutor::getExecutorId,
+                    (Function<IRunningExecutor, ExecutorStats>) IRunningExecutor::renderStats)));
+        }
+        Map<String, Object> zkHB = StatsUtil.mkZkWorkerHb(workerState.topologyId, stats, workerState.uptime.upTime());
+        try {
+            workerState.stormClusterState
+                .workerHeartbeat(workerState.topologyId, workerState.assignmentId, (long) workerState.port,
+                    StatsUtil.thriftifyZkWorkerHb(zkHB));
+        } catch (Exception ex) {
+            LOG.error("Worker failed to write heartbeats to ZK or Pacemaker...will retry", ex);
+        }
+    }
+
+    public void checkCredentialsChanged() {
+        Credentials newCreds = workerState.stormClusterState.credentials(topologyId, null);
+        if (! ObjectUtils.equals(newCreds, credentialsAtom.get())) {
+            // This does not have to be atomic, worst case we update when one is not needed
+            AuthUtils.updateSubject(subject, autoCreds, (null == newCreds) ? null : newCreds.get_creds());
+            for (IRunningExecutor executor : executorsAtom.get()) {
+                executor.credentialsChanged(newCreds);
+            }
+            credentialsAtom.set(newCreds);
+        }
+    }
+
+    public void checkThrottleChanged() {
+        boolean throttleOn = workerState.stormClusterState.topologyBackpressure(topologyId, this::checkThrottleChanged);
+        workerState.throttleOn.set(throttleOn);
+    }
+
+    public void checkLogConfigChanged() {
+        LogConfig logConfig = workerState.stormClusterState.topologyLogConfig(topologyId, null);
+        logConfigManager.processLogConfigChange(logConfig);
+        establishLogSettingCallback();
+    }
+
+    public void establishLogSettingCallback() {
+        workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
+    }
+
+
+    /**
+     * make a handler for the worker's send disruptor queue to
+     * check highWaterMark and lowWaterMark for backpressure
+     */
+    private DisruptorBackpressureCallback mkDisruptorBackpressureHandler(WorkerState workerState) {
+        return new DisruptorBackpressureCallback() {
+            @Override public void highWaterMark() throws Exception {
+                LOG.debug("worker {} transfer-queue is congested, checking backpressure state", workerState.workerId);
+                WorkerBackpressureThread.notifyBackpressureChecker(workerState.backpressureTrigger);
+            }
+
+            @Override public void lowWaterMark() throws Exception {
+                LOG.debug("worker {} transfer-queue is not congested, checking backpressure state", workerState.workerId);
+                WorkerBackpressureThread.notifyBackpressureChecker(workerState.backpressureTrigger);
+            }
+        };
+    }
+
+    /**
+     * make a handler that checks and updates worker's backpressure flag
+     */
+    private WorkerBackpressureCallback mkBackpressureHandler() {
+        final List<IRunningExecutor> executors = executorsAtom.get();
+        return new WorkerBackpressureCallback() {
+            @Override public void onEvent(Object obj) {
+                String topologyId = workerState.topologyId;
+                String assignmentId = workerState.assignmentId;
+                int port = workerState.port;
+                IStormClusterState stormClusterState = workerState.stormClusterState;
+                boolean prevBackpressureFlag = workerState.backpressure.get();
+                boolean currBackpressureFlag = prevBackpressureFlag;
+                if (null != executors) {
+                    currBackpressureFlag = workerState.transferQueue.getThrottleOn() || (executors.stream()
+                        .map(IRunningExecutor::getBackPressureFlag).reduce((op1, op2) -> (op1 || op2)).get());
+                }
+
+                if (currBackpressureFlag != prevBackpressureFlag) {
+                    try {
+                        LOG.debug("worker backpressure flag changing from {} to {}", prevBackpressureFlag, currBackpressureFlag);
+                        stormClusterState.workerBackpressure(topologyId, assignmentId, (long) port, currBackpressureFlag);
+                        // doing the local reset after the zk update succeeds is very important to avoid a bad state upon zk exception
+                        workerState.backpressure.set(currBackpressureFlag);
+                    } catch (Exception ex) {
+                        LOG.error("workerBackpressure update failed when connecting to ZK ... will retry", ex);
+                    }
+                }
+            }
+        };
+    }
+
+    @Override public void shutdown() {
+        try {
+            LOG.info("Shutting down worker {} {} {}", topologyId, assignmentId, port);
+
+            for (IConnection socket : workerState.cachedNodeToPortSocket.get().values()) {
+                //this will do best effort flushing since the linger period
+                // was set on creation
+                socket.close();
+            }
+            LOG.info("Terminating messaging context");
+            LOG.info("Shutting down executors");
+            for (IRunningExecutor executor : executorsAtom.get()) {
+                ((ExecutorShutdown) executor).shutdown();
+            }
+            LOG.info("Shut down executors");
+
+            // this is fine because the only time this is shared is when it's a local context,
+            // in which case it's a noop
+            workerState.mqContext.term();
+            LOG.info("Shutting down transfer thread");
+            workerState.transferQueue.haltWithInterrupt();
+
+            transferThread.interrupt();
+            transferThread.join();
+            LOG.info("Shut down transfer thread");
+
+            backpressureThread.terminate();
+            LOG.info("Shut down backpressure thread");
+
+            workerState.heartbeatTimer.close();
+            workerState.refreshConnectionsTimer.close();
+            workerState.refreshCredentialsTimer.close();
+            workerState.refreshBackpressureTimer.close();
+            workerState.refreshActiveTimer.close();
+            workerState.executorHeartbeatTimer.close();
+            workerState.userTimer.close();
+            workerState.refreshLoadTimer.close();
+            workerState.resetLogLevelsTimer.close();
+            workerState.closeResources();
+
+            LOG.info("Trigger any worker shutdown hooks");
+            workerState.runWorkerShutdownHooks();
+
+            workerState.stormClusterState.removeWorkerHeartbeat(topologyId, assignmentId, (long) port);
+            workerState.stormClusterState.removeWorkerBackpressure(topologyId, assignmentId, (long) port);
+            LOG.info("Disconnecting from storm cluster state context");
+            workerState.stormClusterState.disconnect();
+            workerState.stateStorage.close();
+            LOG.info("Shut down worker {} {} {}", topologyId, assignmentId, port);
+        } catch (Exception ex) {
+            throw Utils.wrapInRuntime(ex);
+        }
+
+    }
+
+    @Override public boolean isWaiting() {
+        return workerState.heartbeatTimer.isTimerWaiting()
+            && workerState.refreshConnectionsTimer.isTimerWaiting()
+            && workerState.refreshLoadTimer.isTimerWaiting()
+            && workerState.refreshCredentialsTimer.isTimerWaiting()
+            && workerState.refreshBackpressureTimer.isTimerWaiting()
+            && workerState.refreshActiveTimer.isTimerWaiting()
+            && workerState.executorHeartbeatTimer.isTimerWaiting()
+            && workerState.userTimer.isTimerWaiting();
+    }
+
+    public static void main(String[] args) throws Exception {
+        Preconditions.checkArgument(args.length == 4, "Illegal number of arguemtns. Expected: 4, Actual: " + args.length);
+        String stormId = args[0];
+        String assignmentId = args[1];
+        String portStr = args[2];
+        String workerId = args[3];
+        Map conf = Utils.readStormConfig();
+        Utils.setupDefaultUncaughtExceptionHandler();
+        StormCommon.validateDistributedMode(conf);
+        Worker worker = new Worker(conf, null, stormId, assignmentId, Integer.parseInt(portStr), workerId);
+        worker.start();
+        Utils.addShutdownHookWithForceKillIn1Sec(worker::shutdown);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
new file mode 100644
index 0000000..3913c32
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
@@ -0,0 +1,690 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.daemon.worker;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.IStateStorage;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.cluster.VersionedData;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.supervisor.AdvancedFSOps;
+import org.apache.storm.generated.Assignment;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.InvalidTopologyException;
+import org.apache.storm.generated.NodeInfo;
+import org.apache.storm.generated.StormBase;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.generated.TopologyStatus;
+import org.apache.storm.grouping.Load;
+import org.apache.storm.grouping.LoadMapping;
+import org.apache.storm.hooks.BaseWorkerHook;
+import org.apache.storm.messaging.ConnectionWithStatus;
+import org.apache.storm.messaging.DeserializingConnectionCallback;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.messaging.TransportFactory;
+import org.apache.storm.serialization.KryoTupleSerializer;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.ThriftTopologyUtils;
+import org.apache.storm.utils.TransferDrainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class WorkerState {
+
+    private static final Logger LOG = LoggerFactory.getLogger(WorkerState.class);
+
+    final Map conf;
+    final IContext mqContext;
+
+    public Map getConf() {
+        return conf;
+    }
+
+    public IConnection getReceiver() {
+        return receiver;
+    }
+
+    public String getTopologyId() {
+        return topologyId;
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+    public String getWorkerId() {
+        return workerId;
+    }
+
+    public IStateStorage getStateStorage() {
+        return stateStorage;
+    }
+
+    public AtomicBoolean getIsTopologyActive() {
+        return isTopologyActive;
+    }
+
+    public AtomicReference<Map<String, DebugOptions>> getStormComponentToDebug() {
+        return stormComponentToDebug;
+    }
+
+    public Set<List<Long>> getExecutors() {
+        return executors;
+    }
+
+    public List<Integer> getTaskIds() {
+        return taskIds;
+    }
+
+    public Map getTopologyConf() {
+        return topologyConf;
+    }
+
+    public StormTopology getTopology() {
+        return topology;
+    }
+
+    public StormTopology getSystemTopology() {
+        return systemTopology;
+    }
+
+    public Map<Integer, String> getTaskToComponent() {
+        return taskToComponent;
+    }
+
+    public Map<String, Map<String, Fields>> getComponentToStreamToFields() {
+        return componentToStreamToFields;
+    }
+
+    public Map<String, List<Integer>> getComponentToSortedTasks() {
+        return componentToSortedTasks;
+    }
+
+    public AtomicReference<Map<NodeInfo, IConnection>> getCachedNodeToPortSocket() {
+        return cachedNodeToPortSocket;
+    }
+
+    public Map<List<Long>, DisruptorQueue> getExecutorReceiveQueueMap() {
+        return executorReceiveQueueMap;
+    }
+
+    public Runnable getSuicideCallback() {
+        return suicideCallback;
+    }
+
+    public Utils.UptimeComputer getUptime() {
+        return uptime;
+    }
+
+    public Map<String, Object> getDefaultSharedResources() {
+        return defaultSharedResources;
+    }
+
+    public Map<String, Object> getUserSharedResources() {
+        return userSharedResources;
+    }
+
+    final IConnection receiver;
+    final String topologyId;
+    final String assignmentId;
+    final int port;
+    final String workerId;
+    final IStateStorage stateStorage;
+    final IStormClusterState stormClusterState;
+
+    // when worker bootup, worker will start to setup initial connections to
+    // other workers. When all connection is ready, we will enable this flag
+    // and spout and bolt will be activated.
+    // used in worker only, keep it as atomic
+    final AtomicBoolean isWorkerActive;
+    final AtomicBoolean isTopologyActive;
+    final AtomicReference<Map<String, DebugOptions>> stormComponentToDebug;
+
+    // executors and taskIds running in this worker
+    final Set<List<Long>> executors;
+    final List<Integer> taskIds;
+    final Map topologyConf;
+    final StormTopology topology;
+    final StormTopology systemTopology;
+    final Map<Integer, String> taskToComponent;
+    final Map<String, Map<String, Fields>> componentToStreamToFields;
+    final Map<String, List<Integer>> componentToSortedTasks;
+    final ReentrantReadWriteLock endpointSocketLock;
+    final AtomicReference<Map<Integer, NodeInfo>> cachedTaskToNodePort;
+    final AtomicReference<Map<NodeInfo, IConnection>> cachedNodeToPortSocket;
+    final Map<List<Long>, DisruptorQueue> executorReceiveQueueMap;
+    // executor id is in form [start_task_id end_task_id]
+    // short executor id is start_task_id
+    final Map<Integer, DisruptorQueue> shortExecutorReceiveQueueMap;
+    final Map<Integer, Integer> taskToShortExecutor;
+    final Runnable suicideCallback;
+    final Utils.UptimeComputer uptime;
+    final Map<String, Object> defaultSharedResources;
+    final Map<String, Object> userSharedResources;
+    final LoadMapping loadMapping;
+    final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
+    // Whether this worker is going slow
+    final AtomicBoolean backpressure = new AtomicBoolean(false);
+    // If the transfer queue is backed-up
+    final AtomicBoolean transferBackpressure = new AtomicBoolean(false);
+    // a trigger for synchronization with executors
+    final AtomicBoolean backpressureTrigger = new AtomicBoolean(false);
+    // whether the throttle is activated for spouts
+    final AtomicBoolean throttleOn = new AtomicBoolean(false);
+
+    public LoadMapping getLoadMapping() {
+        return loadMapping;
+    }
+
+    public AtomicReference<Map<String, VersionedData<Assignment>>> getAssignmentVersions() {
+        return assignmentVersions;
+    }
+
+    public AtomicBoolean getBackpressureTrigger() {
+        return backpressureTrigger;
+    }
+
+    public AtomicBoolean getThrottleOn() {
+        return throttleOn;
+    }
+
+    public DisruptorQueue getTransferQueue() {
+        return transferQueue;
+    }
+
+    public StormTimer getUserTimer() {
+        return userTimer;
+    }
+
+    final DisruptorQueue transferQueue;
+
+    // Timers
+    final StormTimer heartbeatTimer = mkHaltingTimer("heartbeat-timer");
+    final StormTimer refreshLoadTimer = mkHaltingTimer("refresh-load-timer");
+    final StormTimer refreshConnectionsTimer = mkHaltingTimer("refresh-connections-timer");
+    final StormTimer refreshCredentialsTimer = mkHaltingTimer("refresh-credentials-timer");
+    final StormTimer resetLogLevelsTimer = mkHaltingTimer("reset-log-levels-timer");
+    final StormTimer refreshActiveTimer = mkHaltingTimer("refresh-active-timer");
+    final StormTimer executorHeartbeatTimer = mkHaltingTimer("executor-heartbeat-timer");
+    final StormTimer refreshBackpressureTimer = mkHaltingTimer("refresh-backpressure-timer");
+    final StormTimer userTimer = mkHaltingTimer("user-timer");
+
+    // global variables only used internally in class
+    private final Set<Integer> outboundTasks;
+    private final AtomicLong nextUpdate = new AtomicLong(0);
+    private final boolean trySerializeLocal;
+    private final TransferDrainer drainer;
+
+    private static final long LOAD_REFRESH_INTERVAL_MS = 5000L;
+
+    public WorkerState(Map conf, IContext mqContext, String topologyId, String assignmentId, int port, String workerId,
+        Map topologyConf, IStateStorage stateStorage, IStormClusterState stormClusterState)
+        throws IOException, InvalidTopologyException {
+        this.executors = new HashSet<>(readWorkerExecutors(stormClusterState, topologyId, assignmentId, port));
+        this.transferQueue = new DisruptorQueue("worker-transfer-queue",
+            ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_TRANSFER_BUFFER_SIZE)),
+            (long) topologyConf.get(Config.TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS),
+            ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_DISRUPTOR_BATCH_SIZE)),
+            (long) topologyConf.get(Config.TOPOLOGY_DISRUPTOR_BATCH_TIMEOUT_MILLIS));
+
+        this.conf = conf;
+        this.mqContext = (null != mqContext) ? mqContext : TransportFactory.makeContext(topologyConf);
+        this.receiver = this.mqContext.bind(topologyId, port);
+        this.topologyId = topologyId;
+        this.assignmentId = assignmentId;
+        this.port = port;
+        this.workerId = workerId;
+        this.stateStorage = stateStorage;
+        this.stormClusterState = stormClusterState;
+        this.isWorkerActive = new AtomicBoolean(false);
+        this.isTopologyActive = new AtomicBoolean(false);
+        this.stormComponentToDebug = new AtomicReference<>();
+        this.executorReceiveQueueMap = mkReceiveQueueMap(topologyConf, executors);
+        this.shortExecutorReceiveQueueMap = new HashMap<>();
+        this.taskIds = new ArrayList<>();
+        for (Map.Entry<List<Long>, DisruptorQueue> entry : executorReceiveQueueMap.entrySet()) {
+            this.shortExecutorReceiveQueueMap.put(entry.getKey().get(0).intValue(), entry.getValue());
+            this.taskIds.addAll(StormCommon.executorIdToTasks(entry.getKey()));
+        }
+        Collections.sort(taskIds);
+        this.topologyConf = topologyConf;
+        this.topology = ConfigUtils.readSupervisorTopology(conf, topologyId, AdvancedFSOps.make(conf));
+        this.systemTopology = StormCommon.systemTopology(topologyConf, topology);
+        this.taskToComponent = StormCommon.stormTaskInfo(topology, topologyConf);
+        this.componentToStreamToFields = new HashMap<>();
+        for (String c : ThriftTopologyUtils.getComponentIds(systemTopology)) {
+            Map<String, Fields> streamToFields = new HashMap<>();
+            for (Map.Entry<String, StreamInfo> stream : ThriftTopologyUtils.getComponentCommon(systemTopology, c).get_streams().entrySet()) {
+                streamToFields.put(stream.getKey(), new Fields(stream.getValue().get_output_fields()));
+            }
+            componentToStreamToFields.put(c, streamToFields);
+        }
+        this.componentToSortedTasks = Utils.reverseMap(taskToComponent);
+        this.componentToSortedTasks.values().forEach(Collections::sort);
+        this.endpointSocketLock = new ReentrantReadWriteLock();
+        this.cachedNodeToPortSocket = new AtomicReference<>(new HashMap<>());
+        this.cachedTaskToNodePort = new AtomicReference<>(new HashMap<>());
+        this.taskToShortExecutor = new HashMap<>();
+        for (List<Long> executor : this.executors) {
+            for (Integer task : StormCommon.executorIdToTasks(executor)) {
+                taskToShortExecutor.put(task, executor.get(0).intValue());
+            }
+        }
+        this.suicideCallback = Utils.mkSuicideFn();
+        this.uptime = Utils.makeUptimeComputer();
+        this.defaultSharedResources = makeDefaultResources();
+        this.userSharedResources = makeUserResources();
+        this.loadMapping = new LoadMapping();
+        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
+        this.outboundTasks = workerOutboundTasks();
+        this.trySerializeLocal = topologyConf.containsKey(Config.TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE)
+            && (Boolean) topologyConf.get(Config.TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE);
+        if (trySerializeLocal) {
+            LOG.warn("WILL TRY TO SERIALIZE ALL TUPLES (Turn off {} for production", Config.TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE);
+        }
+        this.drainer = new TransferDrainer();
+    }
+
+    public void refreshConnections() {
+        try {
+            refreshConnections(() -> refreshConnectionsTimer.schedule(0, this::refreshConnections));
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    public void refreshConnections(Runnable callback) throws Exception {
+        Integer version = stormClusterState.assignmentVersion(topologyId, callback);
+        version = (null == version) ? 0 : version;
+        VersionedData<Assignment> assignmentVersion = assignmentVersions.get().get(topologyId);
+        Assignment assignment;
+        if (null != assignmentVersion && (assignmentVersion.getVersion() == version)) {
+            assignment = assignmentVersion.getData();
+        } else {
+            VersionedData<Assignment>
+                newAssignmentVersion = new VersionedData<>(version,
+                stormClusterState.assignmentInfoWithVersion(topologyId, callback).getData());
+            assignmentVersions.getAndUpdate(prev -> {
+                Map<String, VersionedData<Assignment>> next = new HashMap<>(prev);
+                next.put(topologyId, newAssignmentVersion);
+                return next;
+            });
+            assignment = newAssignmentVersion.getData();
+        }
+
+        Set<NodeInfo> neededConnections = new HashSet<>();
+        Map<Integer, NodeInfo> newTaskToNodePort = new HashMap<>();
+        if (null != assignment) {
+            Map<Integer, NodeInfo> taskToNodePort = StormCommon.taskToNodeport(assignment.get_executor_node_port());
+            for (Map.Entry<Integer, NodeInfo> taskToNodePortEntry : taskToNodePort.entrySet()) {
+                Integer task = taskToNodePortEntry.getKey();
+                if (outboundTasks.contains(task)) {
+                    newTaskToNodePort.put(task, taskToNodePortEntry.getValue());
+                    if (!taskIds.contains(task)) {
+                        neededConnections.add(taskToNodePortEntry.getValue());
+                    }
+                }
+            }
+        }
+
+        Set<NodeInfo> currentConnections = cachedNodeToPortSocket.get().keySet();
+        Set<NodeInfo> newConnections = Sets.difference(neededConnections, currentConnections);
+        Set<NodeInfo> removeConnections = Sets.difference(currentConnections, neededConnections);
+
+        // Add new connections atomically
+        cachedNodeToPortSocket.getAndUpdate(prev -> {
+            Map<NodeInfo, IConnection> next = new HashMap<>(prev);
+            for (NodeInfo nodeInfo : newConnections) {
+                next.put(nodeInfo,
+                    mqContext.connect(
+                        topologyId,
+                        assignment.get_node_host().get(nodeInfo.get_node()),    // Host
+                        nodeInfo.get_port().iterator().next().intValue()));     // Port
+            }
+            return next;
+        });
+
+
+        try {
+            endpointSocketLock.writeLock().lock();
+            cachedTaskToNodePort.set(newTaskToNodePort);
+        } finally {
+            endpointSocketLock.writeLock().unlock();
+        }
+
+        for (NodeInfo nodeInfo : removeConnections) {
+            cachedNodeToPortSocket.get().get(nodeInfo).close();
+        }
+
+        // Remove old connections atomically
+        cachedNodeToPortSocket.getAndUpdate(prev -> {
+            Map<NodeInfo, IConnection> next = new HashMap<>(prev);
+            removeConnections.forEach(next::remove);
+            return next;
+        });
+
+    }
+
+    public void refreshStormActive() {
+        refreshStormActive(() -> refreshActiveTimer.schedule(0, this::refreshStormActive));
+    }
+
+    public void refreshStormActive(Runnable callback) {
+        StormBase base = stormClusterState.stormBase(topologyId, callback);
+        isTopologyActive.set(
+            (null != base) &&
+            (base.get_status() == TopologyStatus.ACTIVE) &&
+            (isWorkerActive.get()));
+        if (null != base) {
+            Map<String, DebugOptions> debugOptionsMap = new HashMap<>(base.get_component_debug());
+            for (DebugOptions debugOptions : debugOptionsMap.values()) {
+                if (!debugOptions.is_set_samplingpct()) {
+                    debugOptions.set_samplingpct(10);
+                }
+                if (!debugOptions.is_set_enable()) {
+                    debugOptions.set_enable(false);
+                }
+            }
+            stormComponentToDebug.set(debugOptionsMap);
+            LOG.debug("Events debug options {}", stormComponentToDebug.get());
+        }
+    }
+
+    public void refreshThrottle() {
+        boolean backpressure = stormClusterState.topologyBackpressure(topologyId, this::refreshThrottle);
+        this.throttleOn.set(backpressure);
+    }
+
+    public void refreshLoad() {
+        Set<Integer> remoteTasks = Sets.difference(new HashSet<Integer>(outboundTasks), new HashSet<>(taskIds));
+        Long now = System.currentTimeMillis();
+        Map<Integer, Double> localLoad = shortExecutorReceiveQueueMap.entrySet().stream().collect(Collectors.toMap(
+            (Function<Map.Entry<Integer, DisruptorQueue>, Integer>) Map.Entry::getKey,
+            (Function<Map.Entry<Integer, DisruptorQueue>, Double>) entry -> {
+                DisruptorQueue.QueueMetrics qMetrics = entry.getValue().getMetrics();
+                return ( (double) qMetrics.population()) / qMetrics.capacity();
+            }));
+
+        Map<Integer, Load> remoteLoad = new HashMap<>();
+        cachedNodeToPortSocket.get().values().stream().forEach(conn -> remoteLoad.putAll(conn.getLoad(remoteTasks)));
+        loadMapping.setLocal(localLoad);
+        loadMapping.setRemote(remoteLoad);
+
+        if (now > nextUpdate.get()) {
+            receiver.sendLoadMetrics(localLoad);
+            nextUpdate.set(now + LOAD_REFRESH_INTERVAL_MS);
+        }
+    }
+
+    /**
+     * we will wait all connections to be ready and then activate the spout/bolt
+     * when the worker bootup
+     */
+    public void activateWorkerWhenAllConnectionsReady() {
+        int delaySecs = 0;
+        int recurSecs = 1;
+        refreshActiveTimer.schedule(delaySecs, new Runnable() {
+            @Override public void run() {
+                if (areAllConnectionsReady()) {
+                    LOG.info("All connections are ready for worker {}:{} with id", assignmentId, port, workerId);
+                    isWorkerActive.set(Boolean.TRUE);
+                } else {
+                    refreshActiveTimer.schedule(recurSecs, () -> activateWorkerWhenAllConnectionsReady(), false, 0);
+                }
+            }
+        });
+    }
+
+    public void registerCallbacks() {
+        LOG.info("Registering IConnectionCallbacks for {}:{}", assignmentId, port);
+        receiver.registerRecv(new DeserializingConnectionCallback(topologyConf,
+            getWorkerTopologyContext(),
+            this::transferLocal));
+    }
+
+    public void transferLocal(List<AddressedTuple> tupleBatch) {
+        Map<Integer, List<AddressedTuple>> grouped = new HashMap<>();
+        for (AddressedTuple tuple : tupleBatch) {
+            Integer executor = taskToShortExecutor.get(tuple.dest);
+            if (null == executor) {
+                LOG.warn("Received invalid messages for unknown tasks. Dropping... ");
+                continue;
+            }
+            List<AddressedTuple> current = grouped.get(executor);
+            if (null == current) {
+                current = new ArrayList<>();
+                grouped.put(executor, current);
+            }
+            current.add(tuple);
+        }
+
+        for (Map.Entry<Integer, List<AddressedTuple>> entry : grouped.entrySet()) {
+            DisruptorQueue queue = shortExecutorReceiveQueueMap.get(entry.getKey());
+            if (null != queue) {
+                queue.publish(entry.getValue());
+            } else {
+                LOG.warn("Received invalid messages for unknown tasks. Dropping... ");
+            }
+        }
+    }
+
+    public void transfer(KryoTupleSerializer serializer, List<AddressedTuple> tupleBatch) {
+        if (trySerializeLocal) {
+            assertCanSerialize(serializer, tupleBatch);
+        }
+        List<AddressedTuple> local = new ArrayList<>();
+        Map<Integer, List<TaskMessage>> remoteMap = new HashMap<>();
+        for (AddressedTuple addressedTuple : tupleBatch) {
+            int destTask = addressedTuple.getDest();
+            if (taskIds.contains(destTask)) {
+                // Local task
+                local.add(addressedTuple);
+            } else {
+                // Using java objects directly to avoid performance issues in java code
+                if (! remoteMap.containsKey(destTask)) {
+                    remoteMap.put(destTask, new ArrayList<>());
+                }
+                remoteMap.get(destTask).add(new TaskMessage(destTask, serializer.serialize(addressedTuple.getTuple())));
+            }
+        }
+
+        if (!local.isEmpty()) {
+            transferLocal(local);
+        }
+        if (!remoteMap.isEmpty()) {
+            transferQueue.publish(remoteMap);
+        }
+    }
+
+    // TODO: consider having a max batch size besides what disruptor does automagically to prevent latency issues
+    public void sendTuplesToRemoteWorker(HashMap<Integer, ArrayList<TaskMessage>> packets, long seqId, boolean batchEnd) {
+        drainer.add(packets);
+        if (batchEnd) {
+            ReentrantReadWriteLock.ReadLock readLock = endpointSocketLock.readLock();
+            try {
+                readLock.lock();
+                drainer.send(cachedTaskToNodePort.get(), cachedNodeToPortSocket.get());
+            } finally {
+                readLock.unlock();
+            }
+            drainer.clear();
+        }
+    }
+
+
+    private void assertCanSerialize(KryoTupleSerializer serializer, List<AddressedTuple> tuples) {
+        // Check that all of the tuples can be serialized by serializing them
+        for (AddressedTuple addressedTuple : tuples) {
+            serializer.serialize(addressedTuple.getTuple());
+        }
+    }
+
+    public WorkerTopologyContext getWorkerTopologyContext() {
+        try {
+            String codeDir = ConfigUtils.supervisorStormResourcesPath(ConfigUtils.supervisorStormDistRoot(conf, topologyId));
+            String pidDir = ConfigUtils.workerPidsRoot(conf, topologyId);
+            return new WorkerTopologyContext(systemTopology, topologyConf, taskToComponent, componentToSortedTasks,
+                componentToStreamToFields, topologyId, codeDir, pidDir, port, taskIds,
+                defaultSharedResources,
+                userSharedResources);
+        } catch (IOException e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    public void runWorkerStartHooks() {
+        WorkerTopologyContext workerContext = getWorkerTopologyContext();
+        for (ByteBuffer hook : topology.get_worker_hooks()) {
+            byte[] hookBytes = Utils.toByteArray(hook);
+            BaseWorkerHook hookObject = Utils.javaDeserialize(hookBytes, BaseWorkerHook.class);
+            hookObject.start(topologyConf, workerContext);
+
+        }
+    }
+
+    public void runWorkerShutdownHooks() {
+        for (ByteBuffer hook : topology.get_worker_hooks()) {
+            byte[] hookBytes = Utils.toByteArray(hook);
+            BaseWorkerHook hookObject = Utils.javaDeserialize(hookBytes, BaseWorkerHook.class);
+            hookObject.shutdown();
+
+        }
+    }
+
+    public void closeResources() {
+        LOG.info("Shutting down default resources");
+        ((ExecutorService) defaultSharedResources.get(WorkerTopologyContext.SHARED_EXECUTOR)).shutdownNow();
+        LOG.info("Shut down default resources");
+    }
+
+    public boolean areAllConnectionsReady() {
+        return cachedNodeToPortSocket.get().values()
+            .stream()
+            .map(WorkerState::isConnectionReady)
+            .reduce((left, right) -> left && right)
+            .orElse(true);
+    }
+
+    public static boolean isConnectionReady(IConnection connection) {
+        return !(connection instanceof ConnectionWithStatus)
+            || ((ConnectionWithStatus) connection).status() == ConnectionWithStatus.Status.Ready;
+    }
+
+    private List<List<Long>> readWorkerExecutors(IStormClusterState stormClusterState, String topologyId, String assignmentId,
+        int port) {
+        LOG.info("Reading assignments");
+        List<List<Long>> executorsAssignedToThisWorker = new ArrayList<>();
+        executorsAssignedToThisWorker.add(Constants.SYSTEM_EXECUTOR_ID);
+        Map<List<Long>, NodeInfo> executorToNodePort =
+            stormClusterState.assignmentInfo(topologyId, null).get_executor_node_port();
+        for (Map.Entry<List<Long>, NodeInfo> entry : executorToNodePort.entrySet()) {
+            NodeInfo nodeInfo = entry.getValue();
+            if (nodeInfo.get_node().equals(assignmentId) && nodeInfo.get_port().iterator().next() == port) {
+                executorsAssignedToThisWorker.add(entry.getKey());
+            }
+        }
+        return executorsAssignedToThisWorker;
+    }
+
+    private Map<List<Long>, DisruptorQueue> mkReceiveQueueMap(Map topologyConf, Set<List<Long>> executors) {
+        Map<List<Long>, DisruptorQueue> receiveQueueMap = new HashMap<>();
+        for (List<Long> executor : executors) {
+            receiveQueueMap.put(executor, new DisruptorQueue("receive-queue",
+                ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE)),
+                (long) topologyConf.get(Config.TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS),
+                ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_DISRUPTOR_BATCH_SIZE)),
+                (long) topologyConf.get(Config.TOPOLOGY_DISRUPTOR_BATCH_TIMEOUT_MILLIS)));
+        }
+        return receiveQueueMap;
+    }
+    
+    private Map<String, Object> makeDefaultResources() {
+        int threadPoolSize = ObjectReader.getInt(conf.get(Config.TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE));
+        return ImmutableMap.of(WorkerTopologyContext.SHARED_EXECUTOR, Executors.newFixedThreadPool(threadPoolSize));
+    }
+    
+    private Map<String, Object> makeUserResources() {
+        /* TODO: need to invoke a hook provided by the topology, giving it a chance to create user resources.
+        * this would be part of the initialization hook
+        * need to separate workertopologycontext into WorkerContext and WorkerUserContext.
+        * actually just do it via interfaces. just need to make sure to hide setResource from tasks
+        */
+        return new HashMap<>();
+    }
+
+    private StormTimer mkHaltingTimer(String name) {
+        return new StormTimer(name, (thread, exception) -> {
+            LOG.error("Error when processing event", exception);
+            Utils.exitProcess(20, "Error when processing an event");
+        });
+    }
+
+    /**
+     *
+     * @return seq of task ids that receive messages from this worker
+     */
+    private Set<Integer> workerOutboundTasks() {
+        WorkerTopologyContext context = getWorkerTopologyContext();
+        Set<String> components = new HashSet<>();
+        for (Integer taskId : taskIds) {
+            for (Map<String, Grouping> value : context.getTargets(context.getComponentId(taskId)).values()) {
+                components.addAll(value.keySet());
+            }
+        }
+
+        Set<Integer> outboundTasks = new HashSet<>();
+
+        for (Map.Entry<String, List<Integer>> entry : Utils.reverseMap(taskToComponent).entrySet()) {
+            if (components.contains(entry.getKey())) {
+                outboundTasks.addAll(entry.getValue());
+            }
+        }
+        return outboundTasks;
+    }
+
+    public interface ILocalTransferCallback {
+        void transfer(List<AddressedTuple> tupleBatch);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java b/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java
new file mode 100644
index 0000000..9ec680c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.dependency;
+
+import org.apache.commons.lang.StringUtils;
+
+import java.util.UUID;
+
+public class DependencyBlobStoreUtils {
+
+    private static final String BLOB_DEPENDENCIES_PREFIX = "dep-";
+
+    public static String generateDependencyBlobKey(String key) {
+        return BLOB_DEPENDENCIES_PREFIX + key;
+    }
+
+    public static String applyUUIDToFileName(String fileName) {
+        String fileNameWithExt = com.google.common.io.Files.getNameWithoutExtension(fileName);
+        String ext = com.google.common.io.Files.getFileExtension(fileName);
+        if (StringUtils.isEmpty(ext)) {
+            fileName = fileName + "-" + UUID.randomUUID();
+        } else {
+            fileName = fileNameWithExt + "-" + UUID.randomUUID() + "." + ext;
+        }
+        return fileName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/dependency/DependencyPropertiesParser.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/dependency/DependencyPropertiesParser.java b/storm-client/src/jvm/org/apache/storm/dependency/DependencyPropertiesParser.java
new file mode 100644
index 0000000..d360ae0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/dependency/DependencyPropertiesParser.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.dependency;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.json.simple.JSONValue;
+import org.json.simple.parser.ParseException;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+public class DependencyPropertiesParser {
+    public List<File> parseJarsProperties(String prop) {
+        if (prop.trim().isEmpty()) {
+            // handle no input
+            return Collections.emptyList();
+        }
+
+        List<String> dependencies = Arrays.asList(prop.split(","));
+        return Lists.transform(dependencies, new Function<String, File>() {
+            @Override
+            public File apply(String filePath) {
+                return new File(filePath);
+            }
+        });
+    }
+
+    public Map<String, File> parseArtifactsProperties(String prop) {
+        try {
+            Map<String, String> parsed = (Map<String, String>) JSONValue.parseWithException(prop);
+            Map<String, File> packages = new LinkedHashMap<>(parsed.size());
+            for (Map.Entry<String, String> artifactToFilePath : parsed.entrySet()) {
+                packages.put(artifactToFilePath.getKey(), new File(artifactToFilePath.getValue()));
+            }
+
+            return packages;
+        } catch (ParseException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java b/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
new file mode 100644
index 0000000..6fac380
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.dependency;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.storm.blobstore.AtomicOutputStream;
+import org.apache.storm.blobstore.ClientBlobStore;
+import org.apache.storm.generated.AccessControl;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.KeyAlreadyExistsException;
+import org.apache.storm.generated.KeyNotFoundException;
+import org.apache.storm.generated.SettableBlobMeta;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+public class DependencyUploader {
+    public static final Logger LOG = LoggerFactory.getLogger(DependencyUploader.class);
+
+    private final Map<String, Object> conf;
+    private ClientBlobStore blobStore;
+
+    public DependencyUploader() {
+        conf = Utils.readStormConfig();
+    }
+
+    public void init() {
+        //NOOP
+    }
+
+    public void shutdown() {
+        if (blobStore != null) {
+            blobStore.shutdown();
+        }
+    }
+
+    @VisibleForTesting
+    void setBlobStore(ClientBlobStore blobStore) {
+        this.blobStore = blobStore;
+    }
+
+    private synchronized ClientBlobStore getBlobStore() {
+        if (blobStore == null) {
+            blobStore = Utils.getClientBlobStore(conf);
+        }
+        return blobStore;
+    }
+
+    public List<String> uploadFiles(List<File> dependencies, boolean cleanupIfFails) throws IOException, AuthorizationException {
+        checkFilesExist(dependencies);
+
+        List<String> keys = new ArrayList<>(dependencies.size());
+        try {
+            for (File dependency : dependencies) {
+                String fileName = dependency.getName();
+                String key = DependencyBlobStoreUtils.generateDependencyBlobKey(DependencyBlobStoreUtils.applyUUIDToFileName(fileName));
+
+                try {
+                    uploadDependencyToBlobStore(key, dependency);
+                } catch (KeyAlreadyExistsException e) {
+                    // it should never happened since we apply UUID
+                    throw new RuntimeException(e);
+                }
+
+                keys.add(key);
+            }
+        } catch (Throwable e) {
+            if (getBlobStore() != null && cleanupIfFails) {
+                deleteBlobs(keys);
+            }
+            throw new RuntimeException(e);
+        }
+
+        return keys;
+    }
+
+    public List<String> uploadArtifacts(Map<String, File> artifacts) {
+        checkFilesExist(artifacts.values());
+
+        List<String> keys = new ArrayList<>(artifacts.size());
+        try {
+            for (Map.Entry<String, File> artifactToFile : artifacts.entrySet()) {
+                String artifact = artifactToFile.getKey();
+                File dependency = artifactToFile.getValue();
+
+                String key = DependencyBlobStoreUtils.generateDependencyBlobKey(convertArtifactToJarFileName(artifact));
+                try {
+                    uploadDependencyToBlobStore(key, dependency);
+                } catch (KeyAlreadyExistsException e) {
+                    // we lose the race, but it doesn't matter
+                }
+
+                keys.add(key);
+            }
+        } catch (Throwable e) {
+            throw new RuntimeException(e);
+        }
+
+        return keys;
+    }
+
+    public void deleteBlobs(List<String> keys) {
+        for (String key : keys) {
+            try {
+                getBlobStore().deleteBlob(key);
+            } catch (Throwable e) {
+                LOG.warn("blob delete failed - key: {} continue...", key);
+            }
+        }
+    }
+
+    private String convertArtifactToJarFileName(String artifact) {
+        return artifact.replace(":", "-") + ".jar";
+    }
+
+    private boolean uploadDependencyToBlobStore(String key, File dependency)
+            throws KeyAlreadyExistsException, AuthorizationException, IOException {
+
+        boolean uploadNew = false;
+        try {
+            // FIXME: we can filter by listKeys() with local blobstore when STORM-1986 is going to be resolved
+            // as a workaround, we call getBlobMeta() for all keys
+            getBlobStore().getBlobMeta(key);
+        } catch (KeyNotFoundException e) {
+            // TODO: do we want to add ACL here?
+            AtomicOutputStream blob = getBlobStore()
+                    .createBlob(key, new SettableBlobMeta(new ArrayList<AccessControl>()));
+            Files.copy(dependency.toPath(), blob);
+            blob.close();
+
+            uploadNew = true;
+        }
+
+        return uploadNew;
+    }
+
+    private void checkFilesExist(Collection<File> dependencies) {
+        for (File dependency : dependencies) {
+            if (!dependency.isFile() || !dependency.exists()) {
+                throw new FileNotAvailableException(dependency.getAbsolutePath());
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/dependency/FileNotAvailableException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/dependency/FileNotAvailableException.java b/storm-client/src/jvm/org/apache/storm/dependency/FileNotAvailableException.java
new file mode 100644
index 0000000..442e470
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/dependency/FileNotAvailableException.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.dependency;
+
+public class FileNotAvailableException extends RuntimeException {
+    public FileNotAvailableException(String fileName) {
+        super(createMessage(fileName));
+    }
+
+    public FileNotAvailableException(String fileName, Throwable cause) {
+        super(createMessage(fileName), cause);
+    }
+
+    private static String createMessage(String fileName) {
+        return "This file is not available: " + fileName;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java b/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java
new file mode 100644
index 0000000..956999e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.drpc;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.storm.generated.DRPCRequest;
+import org.apache.storm.generated.DistributedRPCInvocations;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.DRPCExecutionException;
+import org.apache.storm.security.auth.ThriftClient;
+import org.apache.storm.security.auth.ThriftConnectionType;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DRPCInvocationsClient extends ThriftClient implements DistributedRPCInvocations.Iface {
+    public static final Logger LOG = LoggerFactory.getLogger(DRPCInvocationsClient.class);
+    private final AtomicReference<DistributedRPCInvocations.Client> client = new AtomicReference<>();
+    private String host;
+    private int port;
+
+    public DRPCInvocationsClient(Map conf, String host, int port) throws TTransportException {
+        super(conf, ThriftConnectionType.DRPC_INVOCATIONS, host, port, null);
+        this.host = host;
+        this.port = port;
+        client.set(new DistributedRPCInvocations.Client(_protocol));
+    }
+        
+    public String getHost() {
+        return host;
+    }
+    
+    public int getPort() {
+        return port;
+    }       
+
+    public void reconnectClient() throws TException {
+        if (client.get() == null) {
+            reconnect();
+            client.set(new DistributedRPCInvocations.Client(_protocol));
+        }
+    }
+
+    public boolean isConnected() {
+        return client.get() != null;
+    }
+
+    public void result(String id, String result) throws TException, AuthorizationException {
+        DistributedRPCInvocations.Client c = client.get();
+        try {
+            if (c == null) {
+                throw new TException("Client is not connected...");
+            }
+            c.result(id, result);
+        } catch(AuthorizationException aze) {
+            throw aze;
+        } catch(TException e) {
+            client.compareAndSet(c, null);
+            throw e;
+        }
+    }
+
+    public DRPCRequest fetchRequest(String func) throws TException, AuthorizationException {
+        DistributedRPCInvocations.Client c = client.get();
+        try {
+            if (c == null) {
+                throw new TException("Client is not connected...");
+            }
+            return c.fetchRequest(func);
+        } catch(AuthorizationException aze) {
+            throw aze;
+        } catch(TException e) {
+            client.compareAndSet(c, null);
+            throw e;
+        }
+    }    
+
+    public void failRequest(String id) throws TException, AuthorizationException {
+        DistributedRPCInvocations.Client c = client.get();
+        try {
+            if (c == null) {
+                throw new TException("Client is not connected...");
+            }
+            c.failRequest(id);
+        } catch(AuthorizationException aze) {
+            throw aze;
+        } catch(TException e) {
+            client.compareAndSet(c, null);
+            throw e;
+        }
+    }
+
+    public DistributedRPCInvocations.Client getClient() {
+        return client.get();
+    }
+
+    @Override
+    public void failRequestV2(String id, DRPCExecutionException ex) throws AuthorizationException, TException {
+        DistributedRPCInvocations.Client c = client.get();
+        try {
+            if (c == null) {
+                throw new TException("Client is not connected...");
+            }
+            c.failRequestV2(id, ex);
+        } catch(AuthorizationException aze) {
+            throw aze;
+        } catch(TException e) {
+            client.compareAndSet(c, null);
+            throw e;
+        }
+    }
+}


[21/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java b/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java
new file mode 100644
index 0000000..b48d342
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java
@@ -0,0 +1,676 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment, LocalAssignment._Fields>, java.io.Serializable, Cloneable, Comparable<LocalAssignment> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalAssignment");
+
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)2);
+  private static final org.apache.thrift.protocol.TField RESOURCES_FIELD_DESC = new org.apache.thrift.protocol.TField("resources", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LocalAssignmentStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LocalAssignmentTupleSchemeFactory());
+  }
+
+  private String topology_id; // required
+  private List<ExecutorInfo> executors; // required
+  private WorkerResources resources; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPOLOGY_ID((short)1, "topology_id"),
+    EXECUTORS((short)2, "executors"),
+    RESOURCES((short)3, "resources");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 2: // EXECUTORS
+          return EXECUTORS;
+        case 3: // RESOURCES
+          return RESOURCES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.RESOURCES};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("executors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class))));
+    tmpMap.put(_Fields.RESOURCES, new org.apache.thrift.meta_data.FieldMetaData("resources", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerResources.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LocalAssignment.class, metaDataMap);
+  }
+
+  public LocalAssignment() {
+  }
+
+  public LocalAssignment(
+    String topology_id,
+    List<ExecutorInfo> executors)
+  {
+    this();
+    this.topology_id = topology_id;
+    this.executors = executors;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LocalAssignment(LocalAssignment other) {
+    if (other.is_set_topology_id()) {
+      this.topology_id = other.topology_id;
+    }
+    if (other.is_set_executors()) {
+      List<ExecutorInfo> __this__executors = new ArrayList<ExecutorInfo>(other.executors.size());
+      for (ExecutorInfo other_element : other.executors) {
+        __this__executors.add(new ExecutorInfo(other_element));
+      }
+      this.executors = __this__executors;
+    }
+    if (other.is_set_resources()) {
+      this.resources = new WorkerResources(other.resources);
+    }
+  }
+
+  public LocalAssignment deepCopy() {
+    return new LocalAssignment(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topology_id = null;
+    this.executors = null;
+    this.resources = null;
+  }
+
+  public String get_topology_id() {
+    return this.topology_id;
+  }
+
+  public void set_topology_id(String topology_id) {
+    this.topology_id = topology_id;
+  }
+
+  public void unset_topology_id() {
+    this.topology_id = null;
+  }
+
+  /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_id() {
+    return this.topology_id != null;
+  }
+
+  public void set_topology_id_isSet(boolean value) {
+    if (!value) {
+      this.topology_id = null;
+    }
+  }
+
+  public int get_executors_size() {
+    return (this.executors == null) ? 0 : this.executors.size();
+  }
+
+  public java.util.Iterator<ExecutorInfo> get_executors_iterator() {
+    return (this.executors == null) ? null : this.executors.iterator();
+  }
+
+  public void add_to_executors(ExecutorInfo elem) {
+    if (this.executors == null) {
+      this.executors = new ArrayList<ExecutorInfo>();
+    }
+    this.executors.add(elem);
+  }
+
+  public List<ExecutorInfo> get_executors() {
+    return this.executors;
+  }
+
+  public void set_executors(List<ExecutorInfo> executors) {
+    this.executors = executors;
+  }
+
+  public void unset_executors() {
+    this.executors = null;
+  }
+
+  /** Returns true if field executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executors() {
+    return this.executors != null;
+  }
+
+  public void set_executors_isSet(boolean value) {
+    if (!value) {
+      this.executors = null;
+    }
+  }
+
+  public WorkerResources get_resources() {
+    return this.resources;
+  }
+
+  public void set_resources(WorkerResources resources) {
+    this.resources = resources;
+  }
+
+  public void unset_resources() {
+    this.resources = null;
+  }
+
+  /** Returns true if field resources is set (has been assigned a value) and false otherwise */
+  public boolean is_set_resources() {
+    return this.resources != null;
+  }
+
+  public void set_resources_isSet(boolean value) {
+    if (!value) {
+      this.resources = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topology_id();
+      } else {
+        set_topology_id((String)value);
+      }
+      break;
+
+    case EXECUTORS:
+      if (value == null) {
+        unset_executors();
+      } else {
+        set_executors((List<ExecutorInfo>)value);
+      }
+      break;
+
+    case RESOURCES:
+      if (value == null) {
+        unset_resources();
+      } else {
+        set_resources((WorkerResources)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      return get_topology_id();
+
+    case EXECUTORS:
+      return get_executors();
+
+    case RESOURCES:
+      return get_resources();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPOLOGY_ID:
+      return is_set_topology_id();
+    case EXECUTORS:
+      return is_set_executors();
+    case RESOURCES:
+      return is_set_resources();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LocalAssignment)
+      return this.equals((LocalAssignment)that);
+    return false;
+  }
+
+  public boolean equals(LocalAssignment that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topology_id = true && this.is_set_topology_id();
+    boolean that_present_topology_id = true && that.is_set_topology_id();
+    if (this_present_topology_id || that_present_topology_id) {
+      if (!(this_present_topology_id && that_present_topology_id))
+        return false;
+      if (!this.topology_id.equals(that.topology_id))
+        return false;
+    }
+
+    boolean this_present_executors = true && this.is_set_executors();
+    boolean that_present_executors = true && that.is_set_executors();
+    if (this_present_executors || that_present_executors) {
+      if (!(this_present_executors && that_present_executors))
+        return false;
+      if (!this.executors.equals(that.executors))
+        return false;
+    }
+
+    boolean this_present_resources = true && this.is_set_resources();
+    boolean that_present_resources = true && that.is_set_resources();
+    if (this_present_resources || that_present_resources) {
+      if (!(this_present_resources && that_present_resources))
+        return false;
+      if (!this.resources.equals(that.resources))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topology_id = true && (is_set_topology_id());
+    list.add(present_topology_id);
+    if (present_topology_id)
+      list.add(topology_id);
+
+    boolean present_executors = true && (is_set_executors());
+    list.add(present_executors);
+    if (present_executors)
+      list.add(executors);
+
+    boolean present_resources = true && (is_set_resources());
+    list.add(present_resources);
+    if (present_resources)
+      list.add(resources);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LocalAssignment other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executors()).compareTo(other.is_set_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executors, other.executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_resources()).compareTo(other.is_set_resources());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_resources()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resources, other.resources);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LocalAssignment(");
+    boolean first = true;
+
+    sb.append("topology_id:");
+    if (this.topology_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topology_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executors:");
+    if (this.executors == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executors);
+    }
+    first = false;
+    if (is_set_resources()) {
+      if (!first) sb.append(", ");
+      sb.append("resources:");
+      if (this.resources == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.resources);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_topology_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topology_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executors()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executors' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (resources != null) {
+      resources.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LocalAssignmentStandardSchemeFactory implements SchemeFactory {
+    public LocalAssignmentStandardScheme getScheme() {
+      return new LocalAssignmentStandardScheme();
+    }
+  }
+
+  private static class LocalAssignmentStandardScheme extends StandardScheme<LocalAssignment> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LocalAssignment struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_id = iprot.readString();
+              struct.set_topology_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list714.size);
+                ExecutorInfo _elem715;
+                for (int _i716 = 0; _i716 < _list714.size; ++_i716)
+                {
+                  _elem715 = new ExecutorInfo();
+                  _elem715.read(iprot);
+                  struct.executors.add(_elem715);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // RESOURCES
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.resources = new WorkerResources();
+              struct.resources.read(iprot);
+              struct.set_resources_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LocalAssignment struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topology_id != null) {
+        oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+        oprot.writeString(struct.topology_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.executors != null) {
+        oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
+          for (ExecutorInfo _iter717 : struct.executors)
+          {
+            _iter717.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.resources != null) {
+        if (struct.is_set_resources()) {
+          oprot.writeFieldBegin(RESOURCES_FIELD_DESC);
+          struct.resources.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LocalAssignmentTupleSchemeFactory implements SchemeFactory {
+    public LocalAssignmentTupleScheme getScheme() {
+      return new LocalAssignmentTupleScheme();
+    }
+  }
+
+  private static class LocalAssignmentTupleScheme extends TupleScheme<LocalAssignment> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LocalAssignment struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.topology_id);
+      {
+        oprot.writeI32(struct.executors.size());
+        for (ExecutorInfo _iter718 : struct.executors)
+        {
+          _iter718.write(oprot);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_resources()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_resources()) {
+        struct.resources.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LocalAssignment struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.topology_id = iprot.readString();
+      struct.set_topology_id_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list719 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list719.size);
+        ExecutorInfo _elem720;
+        for (int _i721 = 0; _i721 < _list719.size; ++_i721)
+        {
+          _elem720 = new ExecutorInfo();
+          _elem720.read(iprot);
+          struct.executors.add(_elem720);
+        }
+      }
+      struct.set_executors_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.resources = new WorkerResources();
+        struct.resources.read(iprot);
+        struct.set_resources_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java b/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java
new file mode 100644
index 0000000..4b8c430
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java
@@ -0,0 +1,471 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, LocalStateData._Fields>, java.io.Serializable, Cloneable, Comparable<LocalStateData> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalStateData");
+
+  private static final org.apache.thrift.protocol.TField SERIALIZED_PARTS_FIELD_DESC = new org.apache.thrift.protocol.TField("serialized_parts", org.apache.thrift.protocol.TType.MAP, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LocalStateDataStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LocalStateDataTupleSchemeFactory());
+  }
+
+  private Map<String,ThriftSerializedObject> serialized_parts; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SERIALIZED_PARTS((short)1, "serialized_parts");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SERIALIZED_PARTS
+          return SERIALIZED_PARTS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SERIALIZED_PARTS, new org.apache.thrift.meta_data.FieldMetaData("serialized_parts", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftSerializedObject.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LocalStateData.class, metaDataMap);
+  }
+
+  public LocalStateData() {
+  }
+
+  public LocalStateData(
+    Map<String,ThriftSerializedObject> serialized_parts)
+  {
+    this();
+    this.serialized_parts = serialized_parts;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LocalStateData(LocalStateData other) {
+    if (other.is_set_serialized_parts()) {
+      Map<String,ThriftSerializedObject> __this__serialized_parts = new HashMap<String,ThriftSerializedObject>(other.serialized_parts.size());
+      for (Map.Entry<String, ThriftSerializedObject> other_element : other.serialized_parts.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ThriftSerializedObject other_element_value = other_element.getValue();
+
+        String __this__serialized_parts_copy_key = other_element_key;
+
+        ThriftSerializedObject __this__serialized_parts_copy_value = new ThriftSerializedObject(other_element_value);
+
+        __this__serialized_parts.put(__this__serialized_parts_copy_key, __this__serialized_parts_copy_value);
+      }
+      this.serialized_parts = __this__serialized_parts;
+    }
+  }
+
+  public LocalStateData deepCopy() {
+    return new LocalStateData(this);
+  }
+
+  @Override
+  public void clear() {
+    this.serialized_parts = null;
+  }
+
+  public int get_serialized_parts_size() {
+    return (this.serialized_parts == null) ? 0 : this.serialized_parts.size();
+  }
+
+  public void put_to_serialized_parts(String key, ThriftSerializedObject val) {
+    if (this.serialized_parts == null) {
+      this.serialized_parts = new HashMap<String,ThriftSerializedObject>();
+    }
+    this.serialized_parts.put(key, val);
+  }
+
+  public Map<String,ThriftSerializedObject> get_serialized_parts() {
+    return this.serialized_parts;
+  }
+
+  public void set_serialized_parts(Map<String,ThriftSerializedObject> serialized_parts) {
+    this.serialized_parts = serialized_parts;
+  }
+
+  public void unset_serialized_parts() {
+    this.serialized_parts = null;
+  }
+
+  /** Returns true if field serialized_parts is set (has been assigned a value) and false otherwise */
+  public boolean is_set_serialized_parts() {
+    return this.serialized_parts != null;
+  }
+
+  public void set_serialized_parts_isSet(boolean value) {
+    if (!value) {
+      this.serialized_parts = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SERIALIZED_PARTS:
+      if (value == null) {
+        unset_serialized_parts();
+      } else {
+        set_serialized_parts((Map<String,ThriftSerializedObject>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SERIALIZED_PARTS:
+      return get_serialized_parts();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SERIALIZED_PARTS:
+      return is_set_serialized_parts();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LocalStateData)
+      return this.equals((LocalStateData)that);
+    return false;
+  }
+
+  public boolean equals(LocalStateData that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_serialized_parts = true && this.is_set_serialized_parts();
+    boolean that_present_serialized_parts = true && that.is_set_serialized_parts();
+    if (this_present_serialized_parts || that_present_serialized_parts) {
+      if (!(this_present_serialized_parts && that_present_serialized_parts))
+        return false;
+      if (!this.serialized_parts.equals(that.serialized_parts))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_serialized_parts = true && (is_set_serialized_parts());
+    list.add(present_serialized_parts);
+    if (present_serialized_parts)
+      list.add(serialized_parts);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LocalStateData other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_serialized_parts()).compareTo(other.is_set_serialized_parts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_serialized_parts()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serialized_parts, other.serialized_parts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LocalStateData(");
+    boolean first = true;
+
+    sb.append("serialized_parts:");
+    if (this.serialized_parts == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.serialized_parts);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_serialized_parts()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'serialized_parts' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LocalStateDataStandardSchemeFactory implements SchemeFactory {
+    public LocalStateDataStandardScheme getScheme() {
+      return new LocalStateDataStandardScheme();
+    }
+  }
+
+  private static class LocalStateDataStandardScheme extends StandardScheme<LocalStateData> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LocalStateData struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SERIALIZED_PARTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map704 = iprot.readMapBegin();
+                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map704.size);
+                String _key705;
+                ThriftSerializedObject _val706;
+                for (int _i707 = 0; _i707 < _map704.size; ++_i707)
+                {
+                  _key705 = iprot.readString();
+                  _val706 = new ThriftSerializedObject();
+                  _val706.read(iprot);
+                  struct.serialized_parts.put(_key705, _val706);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_serialized_parts_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LocalStateData struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.serialized_parts != null) {
+        oprot.writeFieldBegin(SERIALIZED_PARTS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.serialized_parts.size()));
+          for (Map.Entry<String, ThriftSerializedObject> _iter708 : struct.serialized_parts.entrySet())
+          {
+            oprot.writeString(_iter708.getKey());
+            _iter708.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LocalStateDataTupleSchemeFactory implements SchemeFactory {
+    public LocalStateDataTupleScheme getScheme() {
+      return new LocalStateDataTupleScheme();
+    }
+  }
+
+  private static class LocalStateDataTupleScheme extends TupleScheme<LocalStateData> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.serialized_parts.size());
+        for (Map.Entry<String, ThriftSerializedObject> _iter709 : struct.serialized_parts.entrySet())
+        {
+          oprot.writeString(_iter709.getKey());
+          _iter709.getValue().write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map710 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map710.size);
+        String _key711;
+        ThriftSerializedObject _val712;
+        for (int _i713 = 0; _i713 < _map710.size; ++_i713)
+        {
+          _key711 = iprot.readString();
+          _val712 = new ThriftSerializedObject();
+          _val712.read(iprot);
+          struct.serialized_parts.put(_key711, _val712);
+        }
+      }
+      struct.set_serialized_parts_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java b/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java
new file mode 100644
index 0000000..f6633a6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java
@@ -0,0 +1,475 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._Fields>, java.io.Serializable, Cloneable, Comparable<LogConfig> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LogConfig");
+
+  private static final org.apache.thrift.protocol.TField NAMED_LOGGER_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("named_logger_level", org.apache.thrift.protocol.TType.MAP, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LogConfigStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LogConfigTupleSchemeFactory());
+  }
+
+  private Map<String,LogLevel> named_logger_level; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAMED_LOGGER_LEVEL((short)2, "named_logger_level");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 2: // NAMED_LOGGER_LEVEL
+          return NAMED_LOGGER_LEVEL;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.NAMED_LOGGER_LEVEL};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAMED_LOGGER_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("named_logger_level", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LogLevel.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LogConfig.class, metaDataMap);
+  }
+
+  public LogConfig() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LogConfig(LogConfig other) {
+    if (other.is_set_named_logger_level()) {
+      Map<String,LogLevel> __this__named_logger_level = new HashMap<String,LogLevel>(other.named_logger_level.size());
+      for (Map.Entry<String, LogLevel> other_element : other.named_logger_level.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        LogLevel other_element_value = other_element.getValue();
+
+        String __this__named_logger_level_copy_key = other_element_key;
+
+        LogLevel __this__named_logger_level_copy_value = new LogLevel(other_element_value);
+
+        __this__named_logger_level.put(__this__named_logger_level_copy_key, __this__named_logger_level_copy_value);
+      }
+      this.named_logger_level = __this__named_logger_level;
+    }
+  }
+
+  public LogConfig deepCopy() {
+    return new LogConfig(this);
+  }
+
+  @Override
+  public void clear() {
+    this.named_logger_level = null;
+  }
+
+  public int get_named_logger_level_size() {
+    return (this.named_logger_level == null) ? 0 : this.named_logger_level.size();
+  }
+
+  public void put_to_named_logger_level(String key, LogLevel val) {
+    if (this.named_logger_level == null) {
+      this.named_logger_level = new HashMap<String,LogLevel>();
+    }
+    this.named_logger_level.put(key, val);
+  }
+
+  public Map<String,LogLevel> get_named_logger_level() {
+    return this.named_logger_level;
+  }
+
+  public void set_named_logger_level(Map<String,LogLevel> named_logger_level) {
+    this.named_logger_level = named_logger_level;
+  }
+
+  public void unset_named_logger_level() {
+    this.named_logger_level = null;
+  }
+
+  /** Returns true if field named_logger_level is set (has been assigned a value) and false otherwise */
+  public boolean is_set_named_logger_level() {
+    return this.named_logger_level != null;
+  }
+
+  public void set_named_logger_level_isSet(boolean value) {
+    if (!value) {
+      this.named_logger_level = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAMED_LOGGER_LEVEL:
+      if (value == null) {
+        unset_named_logger_level();
+      } else {
+        set_named_logger_level((Map<String,LogLevel>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAMED_LOGGER_LEVEL:
+      return get_named_logger_level();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAMED_LOGGER_LEVEL:
+      return is_set_named_logger_level();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LogConfig)
+      return this.equals((LogConfig)that);
+    return false;
+  }
+
+  public boolean equals(LogConfig that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_named_logger_level = true && this.is_set_named_logger_level();
+    boolean that_present_named_logger_level = true && that.is_set_named_logger_level();
+    if (this_present_named_logger_level || that_present_named_logger_level) {
+      if (!(this_present_named_logger_level && that_present_named_logger_level))
+        return false;
+      if (!this.named_logger_level.equals(that.named_logger_level))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_named_logger_level = true && (is_set_named_logger_level());
+    list.add(present_named_logger_level);
+    if (present_named_logger_level)
+      list.add(named_logger_level);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LogConfig other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_named_logger_level()).compareTo(other.is_set_named_logger_level());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_named_logger_level()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.named_logger_level, other.named_logger_level);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LogConfig(");
+    boolean first = true;
+
+    if (is_set_named_logger_level()) {
+      sb.append("named_logger_level:");
+      if (this.named_logger_level == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.named_logger_level);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LogConfigStandardSchemeFactory implements SchemeFactory {
+    public LogConfigStandardScheme getScheme() {
+      return new LogConfigStandardScheme();
+    }
+  }
+
+  private static class LogConfigStandardScheme extends StandardScheme<LogConfig> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LogConfig struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 2: // NAMED_LOGGER_LEVEL
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map774 = iprot.readMapBegin();
+                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map774.size);
+                String _key775;
+                LogLevel _val776;
+                for (int _i777 = 0; _i777 < _map774.size; ++_i777)
+                {
+                  _key775 = iprot.readString();
+                  _val776 = new LogLevel();
+                  _val776.read(iprot);
+                  struct.named_logger_level.put(_key775, _val776);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_named_logger_level_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LogConfig struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.named_logger_level != null) {
+        if (struct.is_set_named_logger_level()) {
+          oprot.writeFieldBegin(NAMED_LOGGER_LEVEL_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.named_logger_level.size()));
+            for (Map.Entry<String, LogLevel> _iter778 : struct.named_logger_level.entrySet())
+            {
+              oprot.writeString(_iter778.getKey());
+              _iter778.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LogConfigTupleSchemeFactory implements SchemeFactory {
+    public LogConfigTupleScheme getScheme() {
+      return new LogConfigTupleScheme();
+    }
+  }
+
+  private static class LogConfigTupleScheme extends TupleScheme<LogConfig> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_named_logger_level()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_named_logger_level()) {
+        {
+          oprot.writeI32(struct.named_logger_level.size());
+          for (Map.Entry<String, LogLevel> _iter779 : struct.named_logger_level.entrySet())
+          {
+            oprot.writeString(_iter779.getKey());
+            _iter779.getValue().write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map780 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map780.size);
+          String _key781;
+          LogLevel _val782;
+          for (int _i783 = 0; _i783 < _map780.size; ++_i783)
+          {
+            _key781 = iprot.readString();
+            _val782 = new LogLevel();
+            _val782.read(iprot);
+            struct.named_logger_level.put(_key781, _val782);
+          }
+        }
+        struct.set_named_logger_level_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/LogLevel.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LogLevel.java b/storm-client/src/jvm/org/apache/storm/generated/LogLevel.java
new file mode 100644
index 0000000..332cabb
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/LogLevel.java
@@ -0,0 +1,836 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LogLevel implements org.apache.thrift.TBase<LogLevel, LogLevel._Fields>, java.io.Serializable, Cloneable, Comparable<LogLevel> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LogLevel");
+
+  private static final org.apache.thrift.protocol.TField ACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("action", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField TARGET_LOG_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("target_log_level", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField RESET_LOG_LEVEL_TIMEOUT_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("reset_log_level_timeout_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField RESET_LOG_LEVEL_TIMEOUT_EPOCH_FIELD_DESC = new org.apache.thrift.protocol.TField("reset_log_level_timeout_epoch", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField RESET_LOG_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("reset_log_level", org.apache.thrift.protocol.TType.STRING, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LogLevelStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LogLevelTupleSchemeFactory());
+  }
+
+  private LogLevelAction action; // required
+  private String target_log_level; // optional
+  private int reset_log_level_timeout_secs; // optional
+  private long reset_log_level_timeout_epoch; // optional
+  private String reset_log_level; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see LogLevelAction
+     */
+    ACTION((short)1, "action"),
+    TARGET_LOG_LEVEL((short)2, "target_log_level"),
+    RESET_LOG_LEVEL_TIMEOUT_SECS((short)3, "reset_log_level_timeout_secs"),
+    RESET_LOG_LEVEL_TIMEOUT_EPOCH((short)4, "reset_log_level_timeout_epoch"),
+    RESET_LOG_LEVEL((short)5, "reset_log_level");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACTION
+          return ACTION;
+        case 2: // TARGET_LOG_LEVEL
+          return TARGET_LOG_LEVEL;
+        case 3: // RESET_LOG_LEVEL_TIMEOUT_SECS
+          return RESET_LOG_LEVEL_TIMEOUT_SECS;
+        case 4: // RESET_LOG_LEVEL_TIMEOUT_EPOCH
+          return RESET_LOG_LEVEL_TIMEOUT_EPOCH;
+        case 5: // RESET_LOG_LEVEL
+          return RESET_LOG_LEVEL;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID = 0;
+  private static final int __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.TARGET_LOG_LEVEL,_Fields.RESET_LOG_LEVEL_TIMEOUT_SECS,_Fields.RESET_LOG_LEVEL_TIMEOUT_EPOCH,_Fields.RESET_LOG_LEVEL};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACTION, new org.apache.thrift.meta_data.FieldMetaData("action", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, LogLevelAction.class)));
+    tmpMap.put(_Fields.TARGET_LOG_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("target_log_level", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.RESET_LOG_LEVEL_TIMEOUT_SECS, new org.apache.thrift.meta_data.FieldMetaData("reset_log_level_timeout_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.RESET_LOG_LEVEL_TIMEOUT_EPOCH, new org.apache.thrift.meta_data.FieldMetaData("reset_log_level_timeout_epoch", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.RESET_LOG_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("reset_log_level", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LogLevel.class, metaDataMap);
+  }
+
+  public LogLevel() {
+  }
+
+  public LogLevel(
+    LogLevelAction action)
+  {
+    this();
+    this.action = action;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LogLevel(LogLevel other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_action()) {
+      this.action = other.action;
+    }
+    if (other.is_set_target_log_level()) {
+      this.target_log_level = other.target_log_level;
+    }
+    this.reset_log_level_timeout_secs = other.reset_log_level_timeout_secs;
+    this.reset_log_level_timeout_epoch = other.reset_log_level_timeout_epoch;
+    if (other.is_set_reset_log_level()) {
+      this.reset_log_level = other.reset_log_level;
+    }
+  }
+
+  public LogLevel deepCopy() {
+    return new LogLevel(this);
+  }
+
+  @Override
+  public void clear() {
+    this.action = null;
+    this.target_log_level = null;
+    set_reset_log_level_timeout_secs_isSet(false);
+    this.reset_log_level_timeout_secs = 0;
+    set_reset_log_level_timeout_epoch_isSet(false);
+    this.reset_log_level_timeout_epoch = 0;
+    this.reset_log_level = null;
+  }
+
+  /**
+   * 
+   * @see LogLevelAction
+   */
+  public LogLevelAction get_action() {
+    return this.action;
+  }
+
+  /**
+   * 
+   * @see LogLevelAction
+   */
+  public void set_action(LogLevelAction action) {
+    this.action = action;
+  }
+
+  public void unset_action() {
+    this.action = null;
+  }
+
+  /** Returns true if field action is set (has been assigned a value) and false otherwise */
+  public boolean is_set_action() {
+    return this.action != null;
+  }
+
+  public void set_action_isSet(boolean value) {
+    if (!value) {
+      this.action = null;
+    }
+  }
+
+  public String get_target_log_level() {
+    return this.target_log_level;
+  }
+
+  public void set_target_log_level(String target_log_level) {
+    this.target_log_level = target_log_level;
+  }
+
+  public void unset_target_log_level() {
+    this.target_log_level = null;
+  }
+
+  /** Returns true if field target_log_level is set (has been assigned a value) and false otherwise */
+  public boolean is_set_target_log_level() {
+    return this.target_log_level != null;
+  }
+
+  public void set_target_log_level_isSet(boolean value) {
+    if (!value) {
+      this.target_log_level = null;
+    }
+  }
+
+  public int get_reset_log_level_timeout_secs() {
+    return this.reset_log_level_timeout_secs;
+  }
+
+  public void set_reset_log_level_timeout_secs(int reset_log_level_timeout_secs) {
+    this.reset_log_level_timeout_secs = reset_log_level_timeout_secs;
+    set_reset_log_level_timeout_secs_isSet(true);
+  }
+
+  public void unset_reset_log_level_timeout_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field reset_log_level_timeout_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_reset_log_level_timeout_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID);
+  }
+
+  public void set_reset_log_level_timeout_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID, value);
+  }
+
+  public long get_reset_log_level_timeout_epoch() {
+    return this.reset_log_level_timeout_epoch;
+  }
+
+  public void set_reset_log_level_timeout_epoch(long reset_log_level_timeout_epoch) {
+    this.reset_log_level_timeout_epoch = reset_log_level_timeout_epoch;
+    set_reset_log_level_timeout_epoch_isSet(true);
+  }
+
+  public void unset_reset_log_level_timeout_epoch() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID);
+  }
+
+  /** Returns true if field reset_log_level_timeout_epoch is set (has been assigned a value) and false otherwise */
+  public boolean is_set_reset_log_level_timeout_epoch() {
+    return EncodingUtils.testBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID);
+  }
+
+  public void set_reset_log_level_timeout_epoch_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID, value);
+  }
+
+  public String get_reset_log_level() {
+    return this.reset_log_level;
+  }
+
+  public void set_reset_log_level(String reset_log_level) {
+    this.reset_log_level = reset_log_level;
+  }
+
+  public void unset_reset_log_level() {
+    this.reset_log_level = null;
+  }
+
+  /** Returns true if field reset_log_level is set (has been assigned a value) and false otherwise */
+  public boolean is_set_reset_log_level() {
+    return this.reset_log_level != null;
+  }
+
+  public void set_reset_log_level_isSet(boolean value) {
+    if (!value) {
+      this.reset_log_level = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACTION:
+      if (value == null) {
+        unset_action();
+      } else {
+        set_action((LogLevelAction)value);
+      }
+      break;
+
+    case TARGET_LOG_LEVEL:
+      if (value == null) {
+        unset_target_log_level();
+      } else {
+        set_target_log_level((String)value);
+      }
+      break;
+
+    case RESET_LOG_LEVEL_TIMEOUT_SECS:
+      if (value == null) {
+        unset_reset_log_level_timeout_secs();
+      } else {
+        set_reset_log_level_timeout_secs((Integer)value);
+      }
+      break;
+
+    case RESET_LOG_LEVEL_TIMEOUT_EPOCH:
+      if (value == null) {
+        unset_reset_log_level_timeout_epoch();
+      } else {
+        set_reset_log_level_timeout_epoch((Long)value);
+      }
+      break;
+
+    case RESET_LOG_LEVEL:
+      if (value == null) {
+        unset_reset_log_level();
+      } else {
+        set_reset_log_level((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACTION:
+      return get_action();
+
+    case TARGET_LOG_LEVEL:
+      return get_target_log_level();
+
+    case RESET_LOG_LEVEL_TIMEOUT_SECS:
+      return get_reset_log_level_timeout_secs();
+
+    case RESET_LOG_LEVEL_TIMEOUT_EPOCH:
+      return get_reset_log_level_timeout_epoch();
+
+    case RESET_LOG_LEVEL:
+      return get_reset_log_level();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ACTION:
+      return is_set_action();
+    case TARGET_LOG_LEVEL:
+      return is_set_target_log_level();
+    case RESET_LOG_LEVEL_TIMEOUT_SECS:
+      return is_set_reset_log_level_timeout_secs();
+    case RESET_LOG_LEVEL_TIMEOUT_EPOCH:
+      return is_set_reset_log_level_timeout_epoch();
+    case RESET_LOG_LEVEL:
+      return is_set_reset_log_level();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LogLevel)
+      return this.equals((LogLevel)that);
+    return false;
+  }
+
+  public boolean equals(LogLevel that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_action = true && this.is_set_action();
+    boolean that_present_action = true && that.is_set_action();
+    if (this_present_action || that_present_action) {
+      if (!(this_present_action && that_present_action))
+        return false;
+      if (!this.action.equals(that.action))
+        return false;
+    }
+
+    boolean this_present_target_log_level = true && this.is_set_target_log_level();
+    boolean that_present_target_log_level = true && that.is_set_target_log_level();
+    if (this_present_target_log_level || that_present_target_log_level) {
+      if (!(this_present_target_log_level && that_present_target_log_level))
+        return false;
+      if (!this.target_log_level.equals(that.target_log_level))
+        return false;
+    }
+
+    boolean this_present_reset_log_level_timeout_secs = true && this.is_set_reset_log_level_timeout_secs();
+    boolean that_present_reset_log_level_timeout_secs = true && that.is_set_reset_log_level_timeout_secs();
+    if (this_present_reset_log_level_timeout_secs || that_present_reset_log_level_timeout_secs) {
+      if (!(this_present_reset_log_level_timeout_secs && that_present_reset_log_level_timeout_secs))
+        return false;
+      if (this.reset_log_level_timeout_secs != that.reset_log_level_timeout_secs)
+        return false;
+    }
+
+    boolean this_present_reset_log_level_timeout_epoch = true && this.is_set_reset_log_level_timeout_epoch();
+    boolean that_present_reset_log_level_timeout_epoch = true && that.is_set_reset_log_level_timeout_epoch();
+    if (this_present_reset_log_level_timeout_epoch || that_present_reset_log_level_timeout_epoch) {
+      if (!(this_present_reset_log_level_timeout_epoch && that_present_reset_log_level_timeout_epoch))
+        return false;
+      if (this.reset_log_level_timeout_epoch != that.reset_log_level_timeout_epoch)
+        return false;
+    }
+
+    boolean this_present_reset_log_level = true && this.is_set_reset_log_level();
+    boolean that_present_reset_log_level = true && that.is_set_reset_log_level();
+    if (this_present_reset_log_level || that_present_reset_log_level) {
+      if (!(this_present_reset_log_level && that_present_reset_log_level))
+        return false;
+      if (!this.reset_log_level.equals(that.reset_log_level))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_action = true && (is_set_action());
+    list.add(present_action);
+    if (present_action)
+      list.add(action.getValue());
+
+    boolean present_target_log_level = true && (is_set_target_log_level());
+    list.add(present_target_log_level);
+    if (present_target_log_level)
+      list.add(target_log_level);
+
+    boolean present_reset_log_level_timeout_secs = true && (is_set_reset_log_level_timeout_secs());
+    list.add(present_reset_log_level_timeout_secs);
+    if (present_reset_log_level_timeout_secs)
+      list.add(reset_log_level_timeout_secs);
+
+    boolean present_reset_log_level_timeout_epoch = true && (is_set_reset_log_level_timeout_epoch());
+    list.add(present_reset_log_level_timeout_epoch);
+    if (present_reset_log_level_timeout_epoch)
+      list.add(reset_log_level_timeout_epoch);
+
+    boolean present_reset_log_level = true && (is_set_reset_log_level());
+    list.add(present_reset_log_level);
+    if (present_reset_log_level)
+      list.add(reset_log_level);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LogLevel other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_action()).compareTo(other.is_set_action());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_action()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.action, other.action);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_target_log_level()).compareTo(other.is_set_target_log_level());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_target_log_level()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.target_log_level, other.target_log_level);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_reset_log_level_timeout_secs()).compareTo(other.is_set_reset_log_level_timeout_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_reset_log_level_timeout_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reset_log_level_timeout_secs, other.reset_log_level_timeout_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_reset_log_level_timeout_epoch()).compareTo(other.is_set_reset_log_level_timeout_epoch());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_reset_log_level_timeout_epoch()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reset_log_level_timeout_epoch, other.reset_log_level_timeout_epoch);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_reset_log_level()).compareTo(other.is_set_reset_log_level());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_reset_log_level()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reset_log_level, other.reset_log_level);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LogLevel(");
+    boolean first = true;
+
+    sb.append("action:");
+    if (this.action == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.action);
+    }
+    first = false;
+    if (is_set_target_log_level()) {
+      if (!first) sb.append(", ");
+      sb.append("target_log_level:");
+      if (this.target_log_level == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.target_log_level);
+      }
+      first = false;
+    }
+    if (is_set_reset_log_level_timeout_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("reset_log_level_timeout_secs:");
+      sb.append(this.reset_log_level_timeout_secs);
+      first = false;
+    }
+    if (is_set_reset_log_level_timeout_epoch()) {
+      if (!first) sb.append(", ");
+      sb.append("reset_log_level_timeout_epoch:");
+      sb.append(this.reset_log_level_timeout_epoch);
+      first = false;
+    }
+    if (is_set_reset_log_level()) {
+      if (!first) sb.append(", ");
+      sb.append("reset_log_level:");
+      if (this.reset_log_level == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.reset_log_level);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_action()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'action' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LogLevelStandardSchemeFactory implements SchemeFactory {
+    public LogLevelStandardScheme getScheme() {
+      return new LogLevelStandardScheme();
+    }
+  }
+
+  private static class LogLevelStandardScheme extends StandardScheme<LogLevel> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LogLevel struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ACTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.action = org.apache.storm.generated.LogLevelAction.findByValue(iprot.readI32());
+              struct.set_action_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TARGET_LOG_LEVEL
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.target_log_level = iprot.readString();
+              struct.set_target_log_level_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // RESET_LOG_LEVEL_TIMEOUT_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.reset_log_level_timeout_secs = iprot.readI32();
+              struct.set_reset_log_level_timeout_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // RESET_LOG_LEVEL_TIMEOUT_EPOCH
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.reset_log_level_timeout_epoch = iprot.readI64();
+              struct.set_reset_log_level_timeout_epoch_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // RESET_LOG_LEVEL
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.reset_log_level = iprot.readString();
+              struct.set_reset_log_level_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LogLevel struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.action != null) {
+        oprot.writeFieldBegin(ACTION_FIELD_DESC);
+        oprot.writeI32(struct.action.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.target_log_level != null) {
+        if (struct.is_set_target_log_level()) {
+          oprot.writeFieldBegin(TARGET_LOG_LEVEL_FIELD_DESC);
+          oprot.writeString(struct.target_log_level);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_reset_log_level_timeout_secs()) {
+        oprot.writeFieldBegin(RESET_LOG_LEVEL_TIMEOUT_SECS_FIELD_DESC);
+        oprot.writeI32(struct.reset_log_level_timeout_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_reset_log_level_timeout_epoch()) {
+        oprot.writeFieldBegin(RESET_LOG_LEVEL_TIMEOUT_EPOCH_FIELD_DESC);
+        oprot.writeI64(struct.reset_log_level_timeout_epoch);
+        oprot.writeFieldEnd();
+      }
+      if (struct.reset_log_level != null) {
+        if (struct.is_set_reset_log_level()) {
+          oprot.writeFieldBegin(RESET_LOG_LEVEL_FIELD_DESC);
+          oprot.writeString(struct.reset_log_level);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LogLevelTupleSchemeFactory implements SchemeFactory {
+    public LogLevelTupleScheme getScheme() {
+      return new LogLevelTupleScheme();
+    }
+  }
+
+  private static class LogLevelTupleScheme extends TupleScheme<LogLevel> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LogLevel struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.action.getValue());
+      BitSet optionals = new BitSet();
+      if (struct.is_set_target_log_level()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_reset_log_level_timeout_secs()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_reset_log_level_timeout_epoch()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_reset_log_level()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_target_log_level()) {
+        oprot.writeString(struct.target_log_level);
+      }
+      if (struct.is_set_reset_log_level_timeout_secs()) {
+        oprot.writeI32(struct.reset_log_level_timeout_secs);
+      }
+      if (struct.is_set_reset_log_level_timeout_epoch()) {
+        oprot.writeI64(struct.reset_log_level_timeout_epoch);
+      }
+      if (struct.is_set_reset_log_level()) {
+        oprot.writeString(struct.reset_log_level);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LogLevel struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.action = org.apache.storm.generated.LogLevelAction.findByValue(iprot.readI32());
+      struct.set_action_isSet(true);
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.target_log_level = iprot.readString();
+        struct.set_target_log_level_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.reset_log_level_timeout_secs = iprot.readI32();
+        struct.set_reset_log_level_timeout_secs_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.reset_log_level_timeout_epoch = iprot.readI64();
+        struct.set_reset_log_level_timeout_epoch_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.reset_log_level = iprot.readString();
+        struct.set_reset_log_level_isSet(true);
+      }
+    }
+  }
+
+}
+


[07/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologyStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyStats.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyStats.java
new file mode 100644
index 0000000..e63c486
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyStats.java
@@ -0,0 +1,1094 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, TopologyStats._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyStats");
+
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_emitted", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_transferred", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_COMPLETE_LATENCIES_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_complete_latencies_ms", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_acked", org.apache.thrift.protocol.TType.MAP, (short)4);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_failed", org.apache.thrift.protocol.TType.MAP, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologyStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologyStatsTupleSchemeFactory());
+  }
+
+  private Map<String,Long> window_to_emitted; // optional
+  private Map<String,Long> window_to_transferred; // optional
+  private Map<String,Double> window_to_complete_latencies_ms; // optional
+  private Map<String,Long> window_to_acked; // optional
+  private Map<String,Long> window_to_failed; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    WINDOW_TO_EMITTED((short)1, "window_to_emitted"),
+    WINDOW_TO_TRANSFERRED((short)2, "window_to_transferred"),
+    WINDOW_TO_COMPLETE_LATENCIES_MS((short)3, "window_to_complete_latencies_ms"),
+    WINDOW_TO_ACKED((short)4, "window_to_acked"),
+    WINDOW_TO_FAILED((short)5, "window_to_failed");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // WINDOW_TO_EMITTED
+          return WINDOW_TO_EMITTED;
+        case 2: // WINDOW_TO_TRANSFERRED
+          return WINDOW_TO_TRANSFERRED;
+        case 3: // WINDOW_TO_COMPLETE_LATENCIES_MS
+          return WINDOW_TO_COMPLETE_LATENCIES_MS;
+        case 4: // WINDOW_TO_ACKED
+          return WINDOW_TO_ACKED;
+        case 5: // WINDOW_TO_FAILED
+          return WINDOW_TO_FAILED;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.WINDOW_TO_EMITTED,_Fields.WINDOW_TO_TRANSFERRED,_Fields.WINDOW_TO_COMPLETE_LATENCIES_MS,_Fields.WINDOW_TO_ACKED,_Fields.WINDOW_TO_FAILED};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.WINDOW_TO_EMITTED, new org.apache.thrift.meta_data.FieldMetaData("window_to_emitted", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WINDOW_TO_TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("window_to_transferred", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WINDOW_TO_COMPLETE_LATENCIES_MS, new org.apache.thrift.meta_data.FieldMetaData("window_to_complete_latencies_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
+    tmpMap.put(_Fields.WINDOW_TO_ACKED, new org.apache.thrift.meta_data.FieldMetaData("window_to_acked", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WINDOW_TO_FAILED, new org.apache.thrift.meta_data.FieldMetaData("window_to_failed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyStats.class, metaDataMap);
+  }
+
+  public TopologyStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyStats(TopologyStats other) {
+    if (other.is_set_window_to_emitted()) {
+      Map<String,Long> __this__window_to_emitted = new HashMap<String,Long>(other.window_to_emitted);
+      this.window_to_emitted = __this__window_to_emitted;
+    }
+    if (other.is_set_window_to_transferred()) {
+      Map<String,Long> __this__window_to_transferred = new HashMap<String,Long>(other.window_to_transferred);
+      this.window_to_transferred = __this__window_to_transferred;
+    }
+    if (other.is_set_window_to_complete_latencies_ms()) {
+      Map<String,Double> __this__window_to_complete_latencies_ms = new HashMap<String,Double>(other.window_to_complete_latencies_ms);
+      this.window_to_complete_latencies_ms = __this__window_to_complete_latencies_ms;
+    }
+    if (other.is_set_window_to_acked()) {
+      Map<String,Long> __this__window_to_acked = new HashMap<String,Long>(other.window_to_acked);
+      this.window_to_acked = __this__window_to_acked;
+    }
+    if (other.is_set_window_to_failed()) {
+      Map<String,Long> __this__window_to_failed = new HashMap<String,Long>(other.window_to_failed);
+      this.window_to_failed = __this__window_to_failed;
+    }
+  }
+
+  public TopologyStats deepCopy() {
+    return new TopologyStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.window_to_emitted = null;
+    this.window_to_transferred = null;
+    this.window_to_complete_latencies_ms = null;
+    this.window_to_acked = null;
+    this.window_to_failed = null;
+  }
+
+  public int get_window_to_emitted_size() {
+    return (this.window_to_emitted == null) ? 0 : this.window_to_emitted.size();
+  }
+
+  public void put_to_window_to_emitted(String key, long val) {
+    if (this.window_to_emitted == null) {
+      this.window_to_emitted = new HashMap<String,Long>();
+    }
+    this.window_to_emitted.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_emitted() {
+    return this.window_to_emitted;
+  }
+
+  public void set_window_to_emitted(Map<String,Long> window_to_emitted) {
+    this.window_to_emitted = window_to_emitted;
+  }
+
+  public void unset_window_to_emitted() {
+    this.window_to_emitted = null;
+  }
+
+  /** Returns true if field window_to_emitted is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_emitted() {
+    return this.window_to_emitted != null;
+  }
+
+  public void set_window_to_emitted_isSet(boolean value) {
+    if (!value) {
+      this.window_to_emitted = null;
+    }
+  }
+
+  public int get_window_to_transferred_size() {
+    return (this.window_to_transferred == null) ? 0 : this.window_to_transferred.size();
+  }
+
+  public void put_to_window_to_transferred(String key, long val) {
+    if (this.window_to_transferred == null) {
+      this.window_to_transferred = new HashMap<String,Long>();
+    }
+    this.window_to_transferred.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_transferred() {
+    return this.window_to_transferred;
+  }
+
+  public void set_window_to_transferred(Map<String,Long> window_to_transferred) {
+    this.window_to_transferred = window_to_transferred;
+  }
+
+  public void unset_window_to_transferred() {
+    this.window_to_transferred = null;
+  }
+
+  /** Returns true if field window_to_transferred is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_transferred() {
+    return this.window_to_transferred != null;
+  }
+
+  public void set_window_to_transferred_isSet(boolean value) {
+    if (!value) {
+      this.window_to_transferred = null;
+    }
+  }
+
+  public int get_window_to_complete_latencies_ms_size() {
+    return (this.window_to_complete_latencies_ms == null) ? 0 : this.window_to_complete_latencies_ms.size();
+  }
+
+  public void put_to_window_to_complete_latencies_ms(String key, double val) {
+    if (this.window_to_complete_latencies_ms == null) {
+      this.window_to_complete_latencies_ms = new HashMap<String,Double>();
+    }
+    this.window_to_complete_latencies_ms.put(key, val);
+  }
+
+  public Map<String,Double> get_window_to_complete_latencies_ms() {
+    return this.window_to_complete_latencies_ms;
+  }
+
+  public void set_window_to_complete_latencies_ms(Map<String,Double> window_to_complete_latencies_ms) {
+    this.window_to_complete_latencies_ms = window_to_complete_latencies_ms;
+  }
+
+  public void unset_window_to_complete_latencies_ms() {
+    this.window_to_complete_latencies_ms = null;
+  }
+
+  /** Returns true if field window_to_complete_latencies_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_complete_latencies_ms() {
+    return this.window_to_complete_latencies_ms != null;
+  }
+
+  public void set_window_to_complete_latencies_ms_isSet(boolean value) {
+    if (!value) {
+      this.window_to_complete_latencies_ms = null;
+    }
+  }
+
+  public int get_window_to_acked_size() {
+    return (this.window_to_acked == null) ? 0 : this.window_to_acked.size();
+  }
+
+  public void put_to_window_to_acked(String key, long val) {
+    if (this.window_to_acked == null) {
+      this.window_to_acked = new HashMap<String,Long>();
+    }
+    this.window_to_acked.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_acked() {
+    return this.window_to_acked;
+  }
+
+  public void set_window_to_acked(Map<String,Long> window_to_acked) {
+    this.window_to_acked = window_to_acked;
+  }
+
+  public void unset_window_to_acked() {
+    this.window_to_acked = null;
+  }
+
+  /** Returns true if field window_to_acked is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_acked() {
+    return this.window_to_acked != null;
+  }
+
+  public void set_window_to_acked_isSet(boolean value) {
+    if (!value) {
+      this.window_to_acked = null;
+    }
+  }
+
+  public int get_window_to_failed_size() {
+    return (this.window_to_failed == null) ? 0 : this.window_to_failed.size();
+  }
+
+  public void put_to_window_to_failed(String key, long val) {
+    if (this.window_to_failed == null) {
+      this.window_to_failed = new HashMap<String,Long>();
+    }
+    this.window_to_failed.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_failed() {
+    return this.window_to_failed;
+  }
+
+  public void set_window_to_failed(Map<String,Long> window_to_failed) {
+    this.window_to_failed = window_to_failed;
+  }
+
+  public void unset_window_to_failed() {
+    this.window_to_failed = null;
+  }
+
+  /** Returns true if field window_to_failed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_failed() {
+    return this.window_to_failed != null;
+  }
+
+  public void set_window_to_failed_isSet(boolean value) {
+    if (!value) {
+      this.window_to_failed = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case WINDOW_TO_EMITTED:
+      if (value == null) {
+        unset_window_to_emitted();
+      } else {
+        set_window_to_emitted((Map<String,Long>)value);
+      }
+      break;
+
+    case WINDOW_TO_TRANSFERRED:
+      if (value == null) {
+        unset_window_to_transferred();
+      } else {
+        set_window_to_transferred((Map<String,Long>)value);
+      }
+      break;
+
+    case WINDOW_TO_COMPLETE_LATENCIES_MS:
+      if (value == null) {
+        unset_window_to_complete_latencies_ms();
+      } else {
+        set_window_to_complete_latencies_ms((Map<String,Double>)value);
+      }
+      break;
+
+    case WINDOW_TO_ACKED:
+      if (value == null) {
+        unset_window_to_acked();
+      } else {
+        set_window_to_acked((Map<String,Long>)value);
+      }
+      break;
+
+    case WINDOW_TO_FAILED:
+      if (value == null) {
+        unset_window_to_failed();
+      } else {
+        set_window_to_failed((Map<String,Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case WINDOW_TO_EMITTED:
+      return get_window_to_emitted();
+
+    case WINDOW_TO_TRANSFERRED:
+      return get_window_to_transferred();
+
+    case WINDOW_TO_COMPLETE_LATENCIES_MS:
+      return get_window_to_complete_latencies_ms();
+
+    case WINDOW_TO_ACKED:
+      return get_window_to_acked();
+
+    case WINDOW_TO_FAILED:
+      return get_window_to_failed();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case WINDOW_TO_EMITTED:
+      return is_set_window_to_emitted();
+    case WINDOW_TO_TRANSFERRED:
+      return is_set_window_to_transferred();
+    case WINDOW_TO_COMPLETE_LATENCIES_MS:
+      return is_set_window_to_complete_latencies_ms();
+    case WINDOW_TO_ACKED:
+      return is_set_window_to_acked();
+    case WINDOW_TO_FAILED:
+      return is_set_window_to_failed();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyStats)
+      return this.equals((TopologyStats)that);
+    return false;
+  }
+
+  public boolean equals(TopologyStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_window_to_emitted = true && this.is_set_window_to_emitted();
+    boolean that_present_window_to_emitted = true && that.is_set_window_to_emitted();
+    if (this_present_window_to_emitted || that_present_window_to_emitted) {
+      if (!(this_present_window_to_emitted && that_present_window_to_emitted))
+        return false;
+      if (!this.window_to_emitted.equals(that.window_to_emitted))
+        return false;
+    }
+
+    boolean this_present_window_to_transferred = true && this.is_set_window_to_transferred();
+    boolean that_present_window_to_transferred = true && that.is_set_window_to_transferred();
+    if (this_present_window_to_transferred || that_present_window_to_transferred) {
+      if (!(this_present_window_to_transferred && that_present_window_to_transferred))
+        return false;
+      if (!this.window_to_transferred.equals(that.window_to_transferred))
+        return false;
+    }
+
+    boolean this_present_window_to_complete_latencies_ms = true && this.is_set_window_to_complete_latencies_ms();
+    boolean that_present_window_to_complete_latencies_ms = true && that.is_set_window_to_complete_latencies_ms();
+    if (this_present_window_to_complete_latencies_ms || that_present_window_to_complete_latencies_ms) {
+      if (!(this_present_window_to_complete_latencies_ms && that_present_window_to_complete_latencies_ms))
+        return false;
+      if (!this.window_to_complete_latencies_ms.equals(that.window_to_complete_latencies_ms))
+        return false;
+    }
+
+    boolean this_present_window_to_acked = true && this.is_set_window_to_acked();
+    boolean that_present_window_to_acked = true && that.is_set_window_to_acked();
+    if (this_present_window_to_acked || that_present_window_to_acked) {
+      if (!(this_present_window_to_acked && that_present_window_to_acked))
+        return false;
+      if (!this.window_to_acked.equals(that.window_to_acked))
+        return false;
+    }
+
+    boolean this_present_window_to_failed = true && this.is_set_window_to_failed();
+    boolean that_present_window_to_failed = true && that.is_set_window_to_failed();
+    if (this_present_window_to_failed || that_present_window_to_failed) {
+      if (!(this_present_window_to_failed && that_present_window_to_failed))
+        return false;
+      if (!this.window_to_failed.equals(that.window_to_failed))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_window_to_emitted = true && (is_set_window_to_emitted());
+    list.add(present_window_to_emitted);
+    if (present_window_to_emitted)
+      list.add(window_to_emitted);
+
+    boolean present_window_to_transferred = true && (is_set_window_to_transferred());
+    list.add(present_window_to_transferred);
+    if (present_window_to_transferred)
+      list.add(window_to_transferred);
+
+    boolean present_window_to_complete_latencies_ms = true && (is_set_window_to_complete_latencies_ms());
+    list.add(present_window_to_complete_latencies_ms);
+    if (present_window_to_complete_latencies_ms)
+      list.add(window_to_complete_latencies_ms);
+
+    boolean present_window_to_acked = true && (is_set_window_to_acked());
+    list.add(present_window_to_acked);
+    if (present_window_to_acked)
+      list.add(window_to_acked);
+
+    boolean present_window_to_failed = true && (is_set_window_to_failed());
+    list.add(present_window_to_failed);
+    if (present_window_to_failed)
+      list.add(window_to_failed);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologyStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_window_to_emitted()).compareTo(other.is_set_window_to_emitted());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_emitted()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_emitted, other.window_to_emitted);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_transferred()).compareTo(other.is_set_window_to_transferred());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_transferred()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_transferred, other.window_to_transferred);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_complete_latencies_ms()).compareTo(other.is_set_window_to_complete_latencies_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_complete_latencies_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_complete_latencies_ms, other.window_to_complete_latencies_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_acked()).compareTo(other.is_set_window_to_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_acked()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_acked, other.window_to_acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_failed()).compareTo(other.is_set_window_to_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_failed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_failed, other.window_to_failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyStats(");
+    boolean first = true;
+
+    if (is_set_window_to_emitted()) {
+      sb.append("window_to_emitted:");
+      if (this.window_to_emitted == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_emitted);
+      }
+      first = false;
+    }
+    if (is_set_window_to_transferred()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_transferred:");
+      if (this.window_to_transferred == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_transferred);
+      }
+      first = false;
+    }
+    if (is_set_window_to_complete_latencies_ms()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_complete_latencies_ms:");
+      if (this.window_to_complete_latencies_ms == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_complete_latencies_ms);
+      }
+      first = false;
+    }
+    if (is_set_window_to_acked()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_acked:");
+      if (this.window_to_acked == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_acked);
+      }
+      first = false;
+    }
+    if (is_set_window_to_failed()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_failed:");
+      if (this.window_to_failed == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_failed);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologyStatsStandardSchemeFactory implements SchemeFactory {
+    public TopologyStatsStandardScheme getScheme() {
+      return new TopologyStatsStandardScheme();
+    }
+  }
+
+  private static class TopologyStatsStandardScheme extends StandardScheme<TopologyStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // WINDOW_TO_EMITTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map378 = iprot.readMapBegin();
+                struct.window_to_emitted = new HashMap<String,Long>(2*_map378.size);
+                String _key379;
+                long _val380;
+                for (int _i381 = 0; _i381 < _map378.size; ++_i381)
+                {
+                  _key379 = iprot.readString();
+                  _val380 = iprot.readI64();
+                  struct.window_to_emitted.put(_key379, _val380);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_emitted_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // WINDOW_TO_TRANSFERRED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map382 = iprot.readMapBegin();
+                struct.window_to_transferred = new HashMap<String,Long>(2*_map382.size);
+                String _key383;
+                long _val384;
+                for (int _i385 = 0; _i385 < _map382.size; ++_i385)
+                {
+                  _key383 = iprot.readString();
+                  _val384 = iprot.readI64();
+                  struct.window_to_transferred.put(_key383, _val384);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_transferred_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // WINDOW_TO_COMPLETE_LATENCIES_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map386 = iprot.readMapBegin();
+                struct.window_to_complete_latencies_ms = new HashMap<String,Double>(2*_map386.size);
+                String _key387;
+                double _val388;
+                for (int _i389 = 0; _i389 < _map386.size; ++_i389)
+                {
+                  _key387 = iprot.readString();
+                  _val388 = iprot.readDouble();
+                  struct.window_to_complete_latencies_ms.put(_key387, _val388);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_complete_latencies_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // WINDOW_TO_ACKED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map390 = iprot.readMapBegin();
+                struct.window_to_acked = new HashMap<String,Long>(2*_map390.size);
+                String _key391;
+                long _val392;
+                for (int _i393 = 0; _i393 < _map390.size; ++_i393)
+                {
+                  _key391 = iprot.readString();
+                  _val392 = iprot.readI64();
+                  struct.window_to_acked.put(_key391, _val392);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_acked_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // WINDOW_TO_FAILED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map394 = iprot.readMapBegin();
+                struct.window_to_failed = new HashMap<String,Long>(2*_map394.size);
+                String _key395;
+                long _val396;
+                for (int _i397 = 0; _i397 < _map394.size; ++_i397)
+                {
+                  _key395 = iprot.readString();
+                  _val396 = iprot.readI64();
+                  struct.window_to_failed.put(_key395, _val396);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_failed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.window_to_emitted != null) {
+        if (struct.is_set_window_to_emitted()) {
+          oprot.writeFieldBegin(WINDOW_TO_EMITTED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_emitted.size()));
+            for (Map.Entry<String, Long> _iter398 : struct.window_to_emitted.entrySet())
+            {
+              oprot.writeString(_iter398.getKey());
+              oprot.writeI64(_iter398.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_transferred != null) {
+        if (struct.is_set_window_to_transferred()) {
+          oprot.writeFieldBegin(WINDOW_TO_TRANSFERRED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_transferred.size()));
+            for (Map.Entry<String, Long> _iter399 : struct.window_to_transferred.entrySet())
+            {
+              oprot.writeString(_iter399.getKey());
+              oprot.writeI64(_iter399.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_complete_latencies_ms != null) {
+        if (struct.is_set_window_to_complete_latencies_ms()) {
+          oprot.writeFieldBegin(WINDOW_TO_COMPLETE_LATENCIES_MS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.window_to_complete_latencies_ms.size()));
+            for (Map.Entry<String, Double> _iter400 : struct.window_to_complete_latencies_ms.entrySet())
+            {
+              oprot.writeString(_iter400.getKey());
+              oprot.writeDouble(_iter400.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_acked != null) {
+        if (struct.is_set_window_to_acked()) {
+          oprot.writeFieldBegin(WINDOW_TO_ACKED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_acked.size()));
+            for (Map.Entry<String, Long> _iter401 : struct.window_to_acked.entrySet())
+            {
+              oprot.writeString(_iter401.getKey());
+              oprot.writeI64(_iter401.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_failed != null) {
+        if (struct.is_set_window_to_failed()) {
+          oprot.writeFieldBegin(WINDOW_TO_FAILED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_failed.size()));
+            for (Map.Entry<String, Long> _iter402 : struct.window_to_failed.entrySet())
+            {
+              oprot.writeString(_iter402.getKey());
+              oprot.writeI64(_iter402.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TopologyStatsTupleSchemeFactory implements SchemeFactory {
+    public TopologyStatsTupleScheme getScheme() {
+      return new TopologyStatsTupleScheme();
+    }
+  }
+
+  private static class TopologyStatsTupleScheme extends TupleScheme<TopologyStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TopologyStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_window_to_emitted()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_window_to_transferred()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_window_to_complete_latencies_ms()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_window_to_acked()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_window_to_failed()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.is_set_window_to_emitted()) {
+        {
+          oprot.writeI32(struct.window_to_emitted.size());
+          for (Map.Entry<String, Long> _iter403 : struct.window_to_emitted.entrySet())
+          {
+            oprot.writeString(_iter403.getKey());
+            oprot.writeI64(_iter403.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_transferred()) {
+        {
+          oprot.writeI32(struct.window_to_transferred.size());
+          for (Map.Entry<String, Long> _iter404 : struct.window_to_transferred.entrySet())
+          {
+            oprot.writeString(_iter404.getKey());
+            oprot.writeI64(_iter404.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_complete_latencies_ms()) {
+        {
+          oprot.writeI32(struct.window_to_complete_latencies_ms.size());
+          for (Map.Entry<String, Double> _iter405 : struct.window_to_complete_latencies_ms.entrySet())
+          {
+            oprot.writeString(_iter405.getKey());
+            oprot.writeDouble(_iter405.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_acked()) {
+        {
+          oprot.writeI32(struct.window_to_acked.size());
+          for (Map.Entry<String, Long> _iter406 : struct.window_to_acked.entrySet())
+          {
+            oprot.writeString(_iter406.getKey());
+            oprot.writeI64(_iter406.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_failed()) {
+        {
+          oprot.writeI32(struct.window_to_failed.size());
+          for (Map.Entry<String, Long> _iter407 : struct.window_to_failed.entrySet())
+          {
+            oprot.writeString(_iter407.getKey());
+            oprot.writeI64(_iter407.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TopologyStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(5);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map408 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_emitted = new HashMap<String,Long>(2*_map408.size);
+          String _key409;
+          long _val410;
+          for (int _i411 = 0; _i411 < _map408.size; ++_i411)
+          {
+            _key409 = iprot.readString();
+            _val410 = iprot.readI64();
+            struct.window_to_emitted.put(_key409, _val410);
+          }
+        }
+        struct.set_window_to_emitted_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map412 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_transferred = new HashMap<String,Long>(2*_map412.size);
+          String _key413;
+          long _val414;
+          for (int _i415 = 0; _i415 < _map412.size; ++_i415)
+          {
+            _key413 = iprot.readString();
+            _val414 = iprot.readI64();
+            struct.window_to_transferred.put(_key413, _val414);
+          }
+        }
+        struct.set_window_to_transferred_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TMap _map416 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.window_to_complete_latencies_ms = new HashMap<String,Double>(2*_map416.size);
+          String _key417;
+          double _val418;
+          for (int _i419 = 0; _i419 < _map416.size; ++_i419)
+          {
+            _key417 = iprot.readString();
+            _val418 = iprot.readDouble();
+            struct.window_to_complete_latencies_ms.put(_key417, _val418);
+          }
+        }
+        struct.set_window_to_complete_latencies_ms_isSet(true);
+      }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TMap _map420 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_acked = new HashMap<String,Long>(2*_map420.size);
+          String _key421;
+          long _val422;
+          for (int _i423 = 0; _i423 < _map420.size; ++_i423)
+          {
+            _key421 = iprot.readString();
+            _val422 = iprot.readI64();
+            struct.window_to_acked.put(_key421, _val422);
+          }
+        }
+        struct.set_window_to_acked_isSet(true);
+      }
+      if (incoming.get(4)) {
+        {
+          org.apache.thrift.protocol.TMap _map424 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_failed = new HashMap<String,Long>(2*_map424.size);
+          String _key425;
+          long _val426;
+          for (int _i427 = 0; _i427 < _map424.size; ++_i427)
+          {
+            _key425 = iprot.readString();
+            _val426 = iprot.readI64();
+            struct.window_to_failed.put(_key425, _val426);
+          }
+        }
+        struct.set_window_to_failed_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologyStatus.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyStatus.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyStatus.java
new file mode 100644
index 0000000..dd33d12
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyStatus.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum TopologyStatus implements org.apache.thrift.TEnum {
+  ACTIVE(1),
+  INACTIVE(2),
+  REBALANCING(3),
+  KILLED(4);
+
+  private final int value;
+
+  private TopologyStatus(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TopologyStatus findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return ACTIVE;
+      case 2:
+        return INACTIVE;
+      case 3:
+        return REBALANCING;
+      case 4:
+        return KILLED;
+      default:
+        return null;
+    }
+  }
+}


[46/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/StormSubmitter.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/StormSubmitter.java b/storm-client/src/jvm/org/apache/storm/StormSubmitter.java
new file mode 100644
index 0000000..f49369a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/StormSubmitter.java
@@ -0,0 +1,603 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import com.google.common.collect.Sets;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.storm.blobstore.NimbusBlobStore;
+import org.apache.storm.dependency.DependencyPropertiesParser;
+import org.apache.storm.dependency.DependencyUploader;
+import org.apache.storm.hooks.SubmitterHookException;
+import org.apache.storm.scheduler.resource.ResourceUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.validation.ConfigValidation;
+import org.apache.commons.lang.StringUtils;
+import org.apache.thrift.TException;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.generated.*;
+import org.apache.storm.utils.BufferFileInputStream;
+import org.apache.storm.utils.NimbusClient;
+
+/**
+ * Use this class to submit topologies to run on the Storm cluster. You should run your program
+ * with the "storm jar" command from the command-line, and then use this class to
+ * submit your topologies.
+ */
+public class StormSubmitter {
+    public static final Logger LOG = LoggerFactory.getLogger(StormSubmitter.class);
+
+    private static final int THRIFT_CHUNK_SIZE_BYTES = 307200;
+
+    private static ILocalCluster localNimbus = null;
+
+    private static String generateZookeeperDigestSecretPayload() {
+        return Utils.secureRandomLong() + ":" + Utils.secureRandomLong();
+    }
+
+    public static final Pattern zkDigestPattern = Pattern.compile("\\S+:\\S+");
+
+    public static boolean validateZKDigestPayload(String payload) {
+        if (payload != null) {
+            Matcher m = zkDigestPattern.matcher(payload);
+            return m.matches();
+        }
+        return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static Map prepareZookeeperAuthentication(Map conf) {
+        Map toRet = new HashMap();
+
+        // Is the topology ZooKeeper authentication configuration unset?
+        if (! conf.containsKey(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) ||
+                conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) == null ||
+                !  validateZKDigestPayload((String)
+                    conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD))) {
+
+            String secretPayload = generateZookeeperDigestSecretPayload();
+            toRet.put(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD, secretPayload);
+            LOG.info("Generated ZooKeeper secret payload for MD5-digest: " + secretPayload);
+        }
+
+        // This should always be set to digest.
+        toRet.put(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME, "digest");
+
+        return toRet;
+    }
+
+    private static Map<String,String> populateCredentials(Map conf, Map<String, String> creds) {
+        Map<String,String> ret = new HashMap<>();
+        for (IAutoCredentials autoCred: AuthUtils.GetAutoCredentials(conf)) {
+            LOG.info("Running "+autoCred);
+            autoCred.populateCredentials(ret);
+        }
+        if (creds != null) {
+            ret.putAll(creds);
+        }
+        return ret;
+    }
+
+    /**
+     * Push a new set of credentials to the running topology.
+     * @param name the name of the topology to push credentials to.
+     * @param stormConf the topology-specific configuration, if desired. See {@link Config}.
+     * @param credentials the credentials to push.
+     * @throws AuthorizationException if you are not authorized ot push credentials.
+     * @throws NotAliveException if the topology is not alive
+     * @throws InvalidTopologyException if any other error happens
+     */
+    public static void pushCredentials(String name, Map stormConf, Map<String, String> credentials)
+            throws AuthorizationException, NotAliveException, InvalidTopologyException {
+        stormConf = new HashMap(stormConf);
+        stormConf.putAll(Utils.readCommandLineOpts());
+        Map conf = Utils.readStormConfig();
+        conf.putAll(stormConf);
+        Map<String,String> fullCreds = populateCredentials(conf, credentials);
+        if (fullCreds.isEmpty()) {
+            LOG.warn("No credentials were found to push to " + name);
+            return;
+        }
+        try {
+            if (localNimbus!=null) {
+                LOG.info("Pushing Credentials to topology {} in local mode", name);
+                localNimbus.uploadNewCredentials(name, new Credentials(fullCreds));
+            } else {
+                try(NimbusClient client = NimbusClient.getConfiguredClient(conf)) {
+                    LOG.info("Uploading new credentials to {}", name);
+                    client.getClient().uploadNewCredentials(name, new Credentials(fullCreds));
+                }
+            }
+            LOG.info("Finished pushing creds to topology: {}", name);
+        } catch(TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+
+    /**
+     * Submits a topology to run on the cluster. A topology runs forever or until
+     * explicitly killed.
+     *
+     *
+     * @param name the name of the storm.
+     * @param stormConf the topology-specific configuration. See {@link Config}.
+     * @param topology the processing to execute.
+     * @throws AlreadyAliveException if a topology with this name is already running
+     * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
+     * @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
+     */
+    public static void submitTopology(String name, Map stormConf, StormTopology topology)
+            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
+        submitTopology(name, stormConf, topology, null, null);
+    }
+
+    /**
+     * Submits a topology to run on the cluster. A topology runs forever or until
+     * explicitly killed.
+     *
+     * @param name the name of the storm.
+     * @param stormConf the topology-specific configuration. See {@link Config}.
+     * @param topology the processing to execute.
+     * @param opts to manipulate the starting of the topology.
+     * @throws AlreadyAliveException if a topology with this name is already running
+     * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
+     * @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
+     */
+    public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts)
+            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
+        submitTopology(name, stormConf, topology, opts, null);
+    }
+
+    /**
+     * Submits a topology to run on the cluster as a particular user. A topology runs forever or until
+     * explicitly killed.
+     *
+     * @param name
+     * @param stormConf
+     * @param topology
+     * @param opts
+     * @param progressListener
+     * @param asUser The user as which this topology should be submitted.
+     * @throws AlreadyAliveException
+     * @throws InvalidTopologyException
+     * @throws AuthorizationException
+     * @throws IllegalArgumentException thrown if configs will yield an unschedulable topology. validateConfs validates confs
+     * @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
+     */
+    public static void submitTopologyAs(String name, Map stormConf, StormTopology topology, SubmitOptions opts, ProgressListener progressListener, String asUser)
+            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, IllegalArgumentException {
+        if(!Utils.isValidConf(stormConf)) {
+            throw new IllegalArgumentException("Storm conf is not valid. Must be json-serializable");
+        }
+        stormConf = new HashMap(stormConf);
+        stormConf.putAll(Utils.readCommandLineOpts());
+        Map conf = Utils.readStormConfig();
+        conf.putAll(stormConf);
+        stormConf.putAll(prepareZookeeperAuthentication(conf));
+
+        validateConfs(conf, topology);
+
+        Map<String,String> passedCreds = new HashMap<>();
+        if (opts != null) {
+            Credentials tmpCreds = opts.get_creds();
+            if (tmpCreds != null) {
+                passedCreds = tmpCreds.get_creds();
+            }
+        }
+        Map<String,String> fullCreds = populateCredentials(conf, passedCreds);
+        if (!fullCreds.isEmpty()) {
+            if (opts == null) {
+                opts = new SubmitOptions(TopologyInitialStatus.ACTIVE);
+            }
+            opts.set_creds(new Credentials(fullCreds));
+        }
+        try {
+            if (localNimbus!=null) {
+                LOG.info("Submitting topology " + name + " in local mode");
+                if (opts!=null) {
+                    localNimbus.submitTopologyWithOpts(name, stormConf, topology, opts);
+                } else {
+                    // this is for backwards compatibility
+                    localNimbus.submitTopology(name, stormConf, topology);
+                }
+                LOG.info("Finished submitting topology: " +  name);
+            } else {
+                String serConf = JSONValue.toJSONString(stormConf);
+                try (NimbusClient client = NimbusClient.getConfiguredClientAs(conf, asUser)) {
+                    if (topologyNameExists(name, client)) {
+                        throw new RuntimeException("Topology with name `" + name + "` already exists on cluster");
+                    }
+
+                    // Dependency uploading only makes sense for distributed mode
+                    List<String> jarsBlobKeys = Collections.emptyList();
+                    List<String> artifactsBlobKeys;
+
+                    DependencyUploader uploader = new DependencyUploader();
+                    try {
+                        uploader.init();
+
+                        jarsBlobKeys = uploadDependencyJarsToBlobStore(uploader);
+
+                        artifactsBlobKeys = uploadDependencyArtifactsToBlobStore(uploader);
+                    } catch (Throwable e) {
+                        // remove uploaded jars blobs, not artifacts since they're shared across the cluster
+                        uploader.deleteBlobs(jarsBlobKeys);
+                        uploader.shutdown();
+                        throw e;
+                    }
+
+                    try {
+                        setDependencyBlobsToTopology(topology, jarsBlobKeys, artifactsBlobKeys);
+                        submitTopologyInDistributeMode(name, topology, opts, progressListener, asUser, conf, serConf, client);
+                    } catch (AlreadyAliveException | InvalidTopologyException | AuthorizationException e) {
+                        // remove uploaded jars blobs, not artifacts since they're shared across the cluster
+                        // Note that we don't handle TException to delete jars blobs
+                        // because it's safer to leave some blobs instead of topology not running
+                        uploader.deleteBlobs(jarsBlobKeys);
+                        throw e;
+                    } finally {
+                        uploader.shutdown();
+                    }
+                }
+            }
+        } catch(TException e) {
+            throw new RuntimeException(e);
+        }
+        invokeSubmitterHook(name, asUser, conf, topology);
+
+    }
+
+    private static List<String> uploadDependencyJarsToBlobStore(DependencyUploader uploader) {
+        LOG.info("Uploading dependencies - jars...");
+
+        DependencyPropertiesParser propertiesParser = new DependencyPropertiesParser();
+
+        String depJarsProp = System.getProperty("storm.dependency.jars", "");
+        List<File> depJars = propertiesParser.parseJarsProperties(depJarsProp);
+
+        try {
+            return uploader.uploadFiles(depJars, true);
+        } catch (Throwable e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static List<String> uploadDependencyArtifactsToBlobStore(DependencyUploader uploader) {
+        LOG.info("Uploading dependencies - artifacts...");
+
+        DependencyPropertiesParser propertiesParser = new DependencyPropertiesParser();
+
+        String depArtifactsProp = System.getProperty("storm.dependency.artifacts", "{}");
+        Map<String, File> depArtifacts = propertiesParser.parseArtifactsProperties(depArtifactsProp);
+
+        try {
+            return uploader.uploadArtifacts(depArtifacts);
+        } catch (Throwable e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static void setDependencyBlobsToTopology(StormTopology topology, List<String> jarsBlobKeys, List<String> artifactsBlobKeys) {
+        LOG.info("Dependency Blob keys - jars : {} / artifacts : {}", jarsBlobKeys, artifactsBlobKeys);
+        topology.set_dependency_jars(jarsBlobKeys);
+        topology.set_dependency_artifacts(artifactsBlobKeys);
+    }
+
+    private static void submitTopologyInDistributeMode(String name, StormTopology topology, SubmitOptions opts,
+                                                       ProgressListener progressListener, String asUser, Map conf,
+                                                       String serConf, NimbusClient client) throws TException {
+        try {
+            String jar = submitJarAs(conf, System.getProperty("storm.jar"), progressListener, client);
+            LOG.info("Submitting topology {} in distributed mode with conf {}", name, serConf);
+
+            if (opts != null) {
+                client.getClient().submitTopologyWithOpts(name, jar, serConf, topology, opts);
+            } else {
+                // this is for backwards compatibility
+                client.getClient().submitTopology(name, jar, serConf, topology);
+            }
+            LOG.info("Finished submitting topology: {}", name);
+        } catch (InvalidTopologyException e) {
+            LOG.warn("Topology submission exception: {}", e.get_msg());
+            throw e;
+        } catch (AlreadyAliveException e) {
+            LOG.warn("Topology already alive exception", e);
+            throw e;
+        }
+    }
+
+    /**
+     *
+     * @param name
+     * @param asUser
+     * @param stormConf
+     * @param topology
+     *
+     * @thorws SubmitterHookException This is thrown when any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
+     */
+    private static void invokeSubmitterHook(String name, String asUser, Map stormConf, StormTopology topology) {
+        String submissionNotifierClassName = null;
+        try {
+            if (stormConf.containsKey(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN)) {
+                submissionNotifierClassName = stormConf.get(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN).toString();
+                LOG.info("Initializing the registered ISubmitterHook [{}]", submissionNotifierClassName);
+
+                if(submissionNotifierClassName == null || submissionNotifierClassName.isEmpty()) {
+                    throw new IllegalArgumentException(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN + " property must be a non empty string.");
+                }
+
+                ISubmitterHook submitterHook = (ISubmitterHook) Class.forName(submissionNotifierClassName).newInstance();
+                TopologyInfo topologyInfo = Utils.getTopologyInfo(name, asUser, stormConf);
+                LOG.info("Invoking the registered ISubmitterHook [{}]", submissionNotifierClassName);
+                submitterHook.notify(topologyInfo, stormConf, topology);
+            }
+        } catch (Exception e) {
+            LOG.warn("Error occurred in invoking submitter hook:[{}] ",submissionNotifierClassName, e);
+            throw new SubmitterHookException(e);
+        }
+    }
+
+    /**
+     * Submits a topology to run on the cluster. A topology runs forever or until
+     * explicitly killed.
+     *
+     *
+     * @param name the name of the storm.
+     * @param stormConf the topology-specific configuration. See {@link Config}.
+     * @param topology the processing to execute.
+     * @param opts to manipulate the starting of the topology
+     * @param progressListener to track the progress of the jar upload process
+     * @throws AlreadyAliveException if a topology with this name is already running
+     * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
+     * @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
+     */
+    @SuppressWarnings("unchecked")
+    public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts,
+             ProgressListener progressListener) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
+        submitTopologyAs(name, stormConf, topology, opts, progressListener, null);
+    }
+
+    /**
+     * Submits a topology to run on the cluster with a progress bar. A topology runs forever or until
+     * explicitly killed.
+     *
+     *
+     * @param name the name of the storm.
+     * @param stormConf the topology-specific configuration. See {@link Config}.
+     * @param topology the processing to execute.
+     * @throws AlreadyAliveException if a topology with this name is already running
+     * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
+     */
+
+    public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
+        submitTopologyWithProgressBar(name, stormConf, topology, null);
+    }
+
+    /**
+     * Submits a topology to run on the cluster with a progress bar. A topology runs forever or until
+     * explicitly killed.
+     *
+     *
+     * @param name the name of the storm.
+     * @param stormConf the topology-specific configuration. See {@link Config}.
+     * @param topology the processing to execute.
+     * @param opts to manipulate the starting of the topology
+     * @throws AlreadyAliveException if a topology with this name is already running
+     * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
+     * @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
+     */
+    public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology, SubmitOptions opts) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
+        // show a progress bar so we know we're not stuck (especially on slow connections)
+        submitTopology(name, stormConf, topology, opts, new StormSubmitter.ProgressListener() {
+            @Override
+            public void onStart(String srcFile, String targetFile, long totalBytes) {
+                System.out.printf("Start uploading file '%s' to '%s' (%d bytes)\n", srcFile, targetFile, totalBytes);
+            }
+
+            @Override
+            public void onProgress(String srcFile, String targetFile, long bytesUploaded, long totalBytes) {
+                int length = 50;
+                int p = (int)((length * bytesUploaded) / totalBytes);
+                String progress = StringUtils.repeat("=", p);
+                String todo = StringUtils.repeat(" ", length - p);
+
+                System.out.printf("\r[%s%s] %d / %d", progress, todo, bytesUploaded, totalBytes);
+            }
+
+            @Override
+            public void onCompleted(String srcFile, String targetFile, long totalBytes) {
+                System.out.printf("\nFile '%s' uploaded to '%s' (%d bytes)\n", srcFile, targetFile, totalBytes);
+            }
+        });
+    }
+
+    private static boolean topologyNameExists(String name, NimbusClient client) {
+        try {
+            return !client.getClient().isTopologyNameAllowed(name);
+        } catch(Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static String submitJar(Map conf, ProgressListener listener) {
+        return  submitJar(conf, System.getProperty("storm.jar"), listener);
+    }
+
+    /**
+     * Submit jar file
+     * @param conf the topology-specific configuration. See {@link Config}.
+     * @param localJar file path of the jar file to submit
+     * @return the remote location of the submitted jar
+     */
+    public static String submitJar(Map conf, String localJar) {
+        return submitJar(conf, localJar, null);
+    }
+
+    public static String submitJarAs(Map conf, String localJar, ProgressListener listener, NimbusClient client) {
+        if (localJar == null) {
+            throw new RuntimeException("Must submit topologies using the 'storm' client script so that StormSubmitter knows which jar to upload.");
+        }
+
+        try {
+            String uploadLocation = client.getClient().beginFileUpload();
+            LOG.info("Uploading topology jar " + localJar + " to assigned location: " + uploadLocation);
+            BufferFileInputStream is = new BufferFileInputStream(localJar, THRIFT_CHUNK_SIZE_BYTES);
+
+            long totalSize = new File(localJar).length();
+            if (listener != null) {
+                listener.onStart(localJar, uploadLocation, totalSize);
+            }
+
+            long bytesUploaded = 0;
+            while(true) {
+                byte[] toSubmit = is.read();
+                bytesUploaded += toSubmit.length;
+                if (listener != null) {
+                    listener.onProgress(localJar, uploadLocation, bytesUploaded, totalSize);
+                }
+
+                if(toSubmit.length==0) break;
+                client.getClient().uploadChunk(uploadLocation, ByteBuffer.wrap(toSubmit));
+            }
+            client.getClient().finishFileUpload(uploadLocation);
+
+            if (listener != null) {
+                listener.onCompleted(localJar, uploadLocation, totalSize);
+            }
+
+            LOG.info("Successfully uploaded topology jar to assigned location: " + uploadLocation);
+            return uploadLocation;
+        } catch(Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+    
+    public static String submitJarAs(Map conf, String localJar, ProgressListener listener, String asUser) {
+        if (localJar == null) {
+            throw new RuntimeException("Must submit topologies using the 'storm' client script so that StormSubmitter knows which jar to upload.");
+        }
+
+        try (NimbusClient client = NimbusClient.getConfiguredClientAs(conf, asUser)) {
+            return submitJarAs(conf, localJar, listener, client);
+        }
+    }
+
+    /**
+     * Submit jar file
+     * @param conf the topology-specific configuration. See {@link Config}.
+     * @param localJar file path of the jar file to submit
+     * @param listener progress listener to track the jar file upload
+     * @return the remote location of the submitted jar
+     */
+    public static String submitJar(Map conf, String localJar, ProgressListener listener) {
+        return submitJarAs(conf,localJar, listener, (String)null);
+    }
+
+    /**
+     * Interface use to track progress of file upload
+     */
+    public interface ProgressListener {
+        /**
+         * called before file is uploaded
+         * @param srcFile - jar file to be uploaded
+         * @param targetFile - destination file
+         * @param totalBytes - total number of bytes of the file
+         */
+        public void onStart(String srcFile, String targetFile, long totalBytes);
+
+        /**
+         * called whenever a chunk of bytes is uploaded
+         * @param srcFile - jar file to be uploaded
+         * @param targetFile - destination file
+         * @param bytesUploaded - number of bytes transferred so far
+         * @param totalBytes - total number of bytes of the file
+         */
+        public void onProgress(String srcFile, String targetFile, long bytesUploaded, long totalBytes);
+
+        /**
+         * called when the file is uploaded
+         * @param srcFile - jar file to be uploaded
+         * @param targetFile - destination file
+         * @param totalBytes - total number of bytes of the file
+         */
+        public void onCompleted(String srcFile, String targetFile, long totalBytes);
+    }
+
+    private static void validateConfs(Map<String, Object> stormConf, StormTopology topology) throws IllegalArgumentException, InvalidTopologyException {
+        ConfigValidation.validateFields(stormConf);
+        validateTopologyWorkerMaxHeapSizeMBConfigs(stormConf, topology);
+        Utils.validateTopologyBlobStoreMap(stormConf, getListOfKeysFromBlobStore(stormConf));
+    }
+
+    private static void validateTopologyWorkerMaxHeapSizeMBConfigs(Map<String, Object> stormConf, StormTopology topology) {
+        double largestMemReq = getMaxExecutorMemoryUsageForTopo(topology, stormConf);
+        Double topologyWorkerMaxHeapSize = ObjectReader.getDouble(stormConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB));
+        if(topologyWorkerMaxHeapSize < largestMemReq) {
+            throw new IllegalArgumentException("Topology will not be able to be successfully scheduled: Config TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB="
+                    + ObjectReader.getDouble(stormConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB)) + " < "
+                    + largestMemReq + " (Largest memory requirement of a component in the topology). Perhaps set TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB to a larger amount");
+        }
+    }
+
+    private static double getMaxExecutorMemoryUsageForTopo(StormTopology topology, Map topologyConf) {
+        double largestMemoryOperator = 0.0;
+        for(Map<String, Double> entry : ResourceUtils.getBoltsResources(topology, topologyConf).values()) {
+            double memoryRequirement = entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)
+                    + entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
+            if(memoryRequirement > largestMemoryOperator) {
+                largestMemoryOperator = memoryRequirement;
+            }
+        }
+        for(Map<String, Double> entry : ResourceUtils.getSpoutsResources(topology, topologyConf).values()) {
+            double memoryRequirement = entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)
+                    + entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
+            if(memoryRequirement > largestMemoryOperator) {
+                largestMemoryOperator = memoryRequirement;
+            }
+        }
+        return largestMemoryOperator;
+    }
+
+    private static Set<String> getListOfKeysFromBlobStore(Map<String, Object> stormConf) {
+        try (NimbusBlobStore client = new NimbusBlobStore()) {
+            client.prepare(stormConf);
+            return Sets.newHashSet(client.listKeys());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/StormTimer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/StormTimer.java b/storm-client/src/jvm/org/apache/storm/StormTimer.java
new file mode 100644
index 0000000..4f6a7d5
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/StormTimer.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm;
+
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.Time;
+
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Comparator;
+import java.util.Random;
+import java.util.concurrent.PriorityBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The timer defined in this file is very similar to java.util.Timer, except
+ * it integrates with Storm's time simulation capabilities. This lets us test
+ * code that does asynchronous work on the timer thread
+ */
+
+public class StormTimer implements AutoCloseable {
+
+    public static class QueueEntry {
+        public final Long endTimeMs;
+        public final Runnable func;
+        public final String id;
+
+        public QueueEntry(Long endTimeMs, Runnable func, String id) {
+            this.endTimeMs = endTimeMs;
+            this.func = func;
+            this.id = id;
+        }
+    }
+
+    public static class StormTimerTask extends Thread {
+
+        //initialCapacity set to 11 since its the default inital capacity of PriorityBlockingQueue
+        private PriorityBlockingQueue<QueueEntry> queue = new PriorityBlockingQueue<QueueEntry>(11, new Comparator<QueueEntry>() {
+            @Override
+            public int compare(QueueEntry o1, QueueEntry o2) {
+                return o1.endTimeMs.intValue() - o2.endTimeMs.intValue();
+            }
+        });
+
+        // boolean to indicate whether timer is active
+        private AtomicBoolean active = new AtomicBoolean(false);
+
+        // function to call when timer is killed
+        private Thread.UncaughtExceptionHandler onKill;
+
+        //random number generator
+        private Random random = new Random();
+
+        @Override
+        public void run() {
+            while (this.active.get()) {
+                QueueEntry queueEntry = null;
+                try {
+                    queueEntry = this.queue.peek();
+                    if ((queueEntry != null) && (Time.currentTimeMillis() >= queueEntry.endTimeMs)) {
+                        // It is imperative to not run the function
+                        // inside the timer lock. Otherwise, it is
+                        // possible to deadlock if the fn deals with
+                        // other locks, like the submit lock.
+                        this.queue.remove(queueEntry);
+                        queueEntry.func.run();
+                    } else if (queueEntry != null) {
+                        //  If any events are scheduled, sleep until
+                        // event generation. If any recurring events
+                        // are scheduled then we will always go
+                        // through this branch, sleeping only the
+                        // exact necessary amount of time. We give
+                        // an upper bound, e.g. 1000 millis, to the
+                        // sleeping time, to limit the response time
+                        // for detecting any new event within 1 secs.
+                        Time.sleep(Math.min(1000, (queueEntry.endTimeMs - Time.currentTimeMillis())));
+                    } else {
+                        // Otherwise poll to see if any new event
+                        // was scheduled. This is, in essence, the
+                        // response time for detecting any new event
+                        // schedulings when there are no scheduled
+                        // events.
+                        Time.sleep(1000);
+                    }
+                } catch (Throwable e) {
+                    if (!(Utils.exceptionCauseIsInstanceOf(InterruptedException.class, e))
+                            && !(Utils.exceptionCauseIsInstanceOf(ClosedByInterruptException.class, e))) {
+                        this.onKill.uncaughtException(this, e);
+                        this.setActive(false);
+                    }
+                }
+            }
+        }
+
+        public void setOnKillFunc(Thread.UncaughtExceptionHandler onKill) {
+            this.onKill = onKill;
+        }
+
+        public void setActive(boolean flag) {
+            this.active.set(flag);
+        }
+
+        public boolean isActive() {
+            return this.active.get();
+        }
+
+        public void add(QueueEntry queueEntry) {
+            this.queue.add(queueEntry);
+        }
+    }
+
+    //task to run
+    private StormTimerTask task = new StormTimerTask();
+
+    /**
+     * Makes a Timer in the form of a StormTimerTask Object
+     * @param name name of the timer
+     * @param onKill function to call when timer is killed unexpectedly
+     * @return StormTimerTask object that was initialized
+     */
+    public StormTimer (String name, Thread.UncaughtExceptionHandler onKill) {
+        if (onKill == null) {
+            throw new RuntimeException("onKill func is null!");
+        }
+        if (name == null) {
+            this.task.setName("timer");
+        } else {
+            this.task.setName(name);
+        }
+        this.task.setOnKillFunc(onKill);
+        this.task.setActive(true);
+
+        this.task.setDaemon(true);
+        this.task.setPriority(Thread.MAX_PRIORITY);
+        this.task.start();
+    }
+
+    /**
+     * Schedule a function to be executed in the timer
+     * @param delaySecs the number of seconds to delay before running the function
+     * @param func the function to run
+     * @param checkActive whether to check is the timer is active
+     * @param jitterMs add jitter to the run
+     */
+    public void schedule(int delaySecs, Runnable func, boolean checkActive, int jitterMs) {
+        if (func == null) {
+            throw new RuntimeException("function to schedule is null!");
+        }
+        if (checkActive) {
+            checkActive();
+        }
+        String id = Utils.uuid();
+        long endTimeMs = Time.currentTimeMillis() + Time.secsToMillisLong(delaySecs);
+        if (jitterMs > 0) {
+            endTimeMs = this.task.random.nextInt(jitterMs) + endTimeMs;
+        }
+        task.add(new QueueEntry(endTimeMs, func, id));
+    }
+
+    public void schedule(int delaySecs, Runnable func) {
+        schedule(delaySecs, func, true, 0);
+    }
+
+    /**
+     * Schedule a function to run recurrently
+     * @param delaySecs the number of seconds to delay before running the function
+     * @param recurSecs the time between each invocation
+     * @param func the function to run
+     */
+    public void scheduleRecurring(int delaySecs, final int recurSecs, final Runnable func) {
+        schedule(delaySecs, new Runnable() {
+            @Override
+            public void run() {
+                func.run();
+                // This avoids a race condition with cancel-timer.
+                schedule(recurSecs, this, false, 0);
+            }
+        });
+    }
+
+    /**
+     * schedule a function to run recurrently with jitter
+     * @param delaySecs the number of seconds to delay before running the function
+     * @param recurSecs the time between each invocation
+     * @param jitterMs jitter added to the run
+     * @param func the function to run
+     */
+    public void scheduleRecurringWithJitter(int delaySecs, final int recurSecs, final int jitterMs, final Runnable func) {
+        schedule(delaySecs, new Runnable() {
+            @Override
+            public void run() {
+                func.run();
+                // This avoids a race condition with cancel-timer.
+                schedule(recurSecs, this, false, jitterMs);
+            }
+        });
+    }
+
+    /**
+     * check if timer is active
+     */
+    private void checkActive() {
+        if (!this.task.isActive()) {
+            throw new IllegalStateException("Timer is not active");
+        }
+    }
+
+    /**
+     * cancel timer
+     */
+
+    @Override
+    public void close() throws Exception {
+        if (this.task.isActive()) {
+            this.task.setActive(false);
+            this.task.interrupt();
+            this.task.join();
+        }
+    }
+
+    /**
+     * is timer waiting. Used in timer simulation
+     */
+    public boolean isTimerWaiting() {
+        return Time.isThreadWaiting(task);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/Thrift.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/Thrift.java b/storm-client/src/jvm/org/apache/storm/Thrift.java
new file mode 100644
index 0000000..2e5080a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/Thrift.java
@@ -0,0 +1,350 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.JavaObjectArg;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StateSpoutSpec;
+import org.apache.storm.generated.StreamInfo;
+
+import java.lang.reflect.Constructor;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.HashMap;
+import java.io.Serializable;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.storm.generated.JavaObject;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.StormTopology._Fields;
+import org.apache.storm.generated.ComponentCommon;
+import org.apache.storm.generated.NullStruct;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.ComponentObject;
+
+import org.apache.storm.task.IBolt;
+import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.SpoutDeclarer;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.storm.topology.TopologyBuilder;
+
+public class Thrift {
+    private static Logger LOG = LoggerFactory.getLogger(Thrift.class);
+
+    private static StormTopology._Fields[] STORM_TOPOLOGY_FIELDS = null;
+    private static StormTopology._Fields[] SPOUT_FIELDS =
+            { StormTopology._Fields.SPOUTS, StormTopology._Fields.STATE_SPOUTS };
+
+    static {
+        Set<_Fields> keys = StormTopology.metaDataMap.keySet();
+        keys.toArray(STORM_TOPOLOGY_FIELDS = new StormTopology._Fields[keys.size()]);
+    }
+
+    public static StormTopology._Fields[] getTopologyFields() {
+        return STORM_TOPOLOGY_FIELDS;
+    }
+
+    public static StormTopology._Fields[] getSpoutFields() {
+        return SPOUT_FIELDS;
+    }
+
+    public static class SpoutDetails {
+        private IRichSpout spout;
+        private Integer parallelism;
+        private Map conf;
+
+        public SpoutDetails(IRichSpout spout, Integer parallelism, Map conf) {
+            this.spout = spout;
+            this.parallelism = parallelism;
+            this.conf = conf;
+        }
+
+        public IRichSpout getSpout() {
+            return spout;
+        }
+
+        public Integer getParallelism() {
+            return parallelism;
+        }
+
+        public Map getConf() {
+            return conf;
+        }
+    }
+
+    public static class BoltDetails {
+        private Object bolt;
+        private Map conf;
+        private Integer parallelism;
+        private Map<GlobalStreamId, Grouping> inputs;
+
+        public BoltDetails(Object bolt, Map conf, Integer parallelism,
+                           Map<GlobalStreamId, Grouping> inputs) {
+            this.bolt = bolt;
+            this.conf = conf;
+            this.parallelism = parallelism;
+            this.inputs = inputs;
+        }
+
+        public Object getBolt() {
+            return bolt;
+        }
+
+        public Map getConf() {
+            return conf;
+        }
+
+        public Map<GlobalStreamId, Grouping> getInputs() {
+            return inputs;
+        }
+
+        public Integer getParallelism() {
+            return parallelism;
+        }
+    }
+
+    public static StreamInfo directOutputFields(List<String> fields) {
+        return new StreamInfo(fields, true);
+    }
+
+    public static StreamInfo outputFields(List<String> fields) {
+        return new StreamInfo(fields, false);
+    }
+
+    public static Grouping prepareShuffleGrouping() {
+        return Grouping.shuffle(new NullStruct());
+    }
+
+    public static Grouping prepareLocalOrShuffleGrouping() {
+        return Grouping.local_or_shuffle(new NullStruct());
+    }
+
+    public static Grouping prepareFieldsGrouping(List<String> fields) {
+        return Grouping.fields(fields);
+    }
+
+    public static Grouping prepareGlobalGrouping() {
+        return prepareFieldsGrouping(new ArrayList<String>());
+    }
+
+    public static Grouping prepareDirectGrouping() {
+        return Grouping.direct(new NullStruct());
+    }
+
+    public static Grouping prepareAllGrouping() {
+        return Grouping.all(new NullStruct());
+    }
+
+    public static Grouping prepareNoneGrouping() {
+        return Grouping.none(new NullStruct());
+    }
+
+    public static Grouping prepareCustomStreamGrouping(Object obj) {
+        return Grouping.custom_serialized(Utils.javaSerialize(obj));
+    }
+
+    public static Grouping prepareCustomJavaObjectGrouping(JavaObject obj) {
+        return Grouping.custom_object(obj);
+    }
+
+    public static Object instantiateJavaObject(JavaObject obj) {
+
+        List<JavaObjectArg> args = obj.get_args_list();
+        Class[] paraTypes = new Class[args.size()];
+        Object[] paraValues = new Object[args.size()];
+        for (int i = 0; i < args.size(); i++) {
+            JavaObjectArg arg = args.get(i);
+            paraValues[i] = arg.getFieldValue();
+
+            if (arg.getSetField().equals(JavaObjectArg._Fields.INT_ARG)) {
+                paraTypes[i] = Integer.class;
+            } else if (arg.getSetField().equals(JavaObjectArg._Fields.LONG_ARG)) {
+                paraTypes[i] = Long.class;
+            } else if (arg.getSetField().equals(JavaObjectArg._Fields.STRING_ARG)) {
+                paraTypes[i] = String.class;
+            } else if (arg.getSetField().equals(JavaObjectArg._Fields.BOOL_ARG)) {
+                paraTypes[i] = Boolean.class;
+            } else if (arg.getSetField().equals(JavaObjectArg._Fields.BINARY_ARG)) {
+                paraTypes[i] = ByteBuffer.class;
+            } else if (arg.getSetField().equals(JavaObjectArg._Fields.DOUBLE_ARG)) {
+                paraTypes[i] = Double.class;
+            } else {
+                paraTypes[i] = Object.class;
+            }
+        }
+
+        try {
+            Class clazz = Class.forName(obj.get_full_class_name());
+            Constructor cons = clazz.getConstructor(paraTypes);
+            return cons.newInstance(paraValues);
+        } catch (Exception e) {
+            LOG.error("java object instantiation failed", e);
+        }
+
+        return null;
+
+    }
+
+    public static Grouping._Fields groupingType(Grouping grouping) {
+        return grouping.getSetField();
+    }
+
+    public static List<String> fieldGrouping(Grouping grouping) {
+        if (!Grouping._Fields.FIELDS.equals(groupingType(grouping))) {
+            throw new IllegalArgumentException("Tried to get grouping fields from non fields grouping");
+        }
+        return grouping.get_fields();
+    }
+
+    public static boolean isGlobalGrouping(Grouping grouping) {
+        if (Grouping._Fields.FIELDS.equals(groupingType(grouping))) {
+            return fieldGrouping(grouping).isEmpty();
+        }
+
+        return false;
+    }
+
+    public static int getParallelismHint(ComponentCommon componentCommon) {
+        if (!componentCommon.is_set_parallelism_hint()) {
+            return 1;
+        } else {
+            return componentCommon.get_parallelism_hint();
+        }
+    }
+
+    public static ComponentObject serializeComponentObject(Object obj) {
+        return ComponentObject.serialized_java(Utils.javaSerialize(obj));
+    }
+
+    public static Object deserializeComponentObject(ComponentObject obj) {
+        if (obj.getSetField() != ComponentObject._Fields.SERIALIZED_JAVA) {
+            throw new RuntimeException("Cannot deserialize non-java-serialized object");
+        }
+        return Utils.javaDeserialize(obj.get_serialized_java(), Serializable.class);
+    }
+
+    public static ComponentCommon prepareComponentCommon(Map<GlobalStreamId, Grouping> inputs, Map<String,
+            StreamInfo> outputs, Integer parallelismHint) {
+        return prepareComponentCommon(inputs, outputs, parallelismHint, null);
+    }
+
+    public static ComponentCommon prepareComponentCommon(Map<GlobalStreamId, Grouping> inputs, Map<String, StreamInfo> outputs,
+                                                         Integer parallelismHint, Map conf) {
+        Map<GlobalStreamId, Grouping> mappedInputs = new HashMap<>();
+        Map<String, StreamInfo> mappedOutputs = new HashMap<>();
+        if (inputs != null && !inputs.isEmpty()) {
+            mappedInputs.putAll(inputs);
+        }
+        if (outputs !=null && !outputs.isEmpty()) {
+            mappedOutputs.putAll(outputs);
+        }
+        ComponentCommon component = new ComponentCommon(mappedInputs, mappedOutputs);
+        if (parallelismHint != null) {
+            component.set_parallelism_hint(parallelismHint);
+        }
+        if (conf != null) {
+            component.set_json_conf(JSONValue.toJSONString(conf));
+        }
+        return component;
+    }
+
+    public static SpoutSpec prepareSerializedSpoutDetails(IRichSpout spout, Map<String, StreamInfo> outputs) {
+        return new SpoutSpec(ComponentObject.serialized_java
+                (Utils.javaSerialize(spout)), prepareComponentCommon(new HashMap<>(), outputs, null, null));
+    }
+
+    public static Bolt prepareSerializedBoltDetails(Map<GlobalStreamId, Grouping> inputs, IBolt bolt, Map<String, StreamInfo> outputs,
+                                                    Integer parallelismHint, Map conf) {
+        ComponentCommon common = prepareComponentCommon(inputs, outputs, parallelismHint, conf);
+        return new Bolt(ComponentObject.serialized_java(Utils.javaSerialize(bolt)), common);
+    }
+
+    public static BoltDetails prepareBoltDetails(Map<GlobalStreamId, Grouping> inputs, Object bolt) {
+        return prepareBoltDetails(inputs, bolt, null, null);
+    }
+
+    public static BoltDetails prepareBoltDetails(Map<GlobalStreamId, Grouping> inputs, Object bolt,
+                                                 Integer parallelismHint) {
+        return prepareBoltDetails(inputs, bolt, parallelismHint, null);
+    }
+
+    public static BoltDetails prepareBoltDetails(Map<GlobalStreamId, Grouping> inputs, Object bolt,
+                                                 Integer parallelismHint, Map conf) {
+        BoltDetails details = new BoltDetails(bolt, conf, parallelismHint, inputs);
+        return details;
+    }
+
+    public static SpoutDetails prepareSpoutDetails(IRichSpout spout) {
+        return prepareSpoutDetails(spout, null, null);
+    }
+
+    public static SpoutDetails prepareSpoutDetails(IRichSpout spout, Integer parallelismHint) {
+        return prepareSpoutDetails(spout, parallelismHint, null);
+    }
+
+    public static SpoutDetails prepareSpoutDetails(IRichSpout spout, Integer parallelismHint, Map conf) {
+        SpoutDetails details = new SpoutDetails(spout, parallelismHint, conf);
+        return details;
+    }
+
+    public static StormTopology buildTopology(HashMap<String, SpoutDetails> spoutMap,
+                                              HashMap<String, BoltDetails> boltMap, HashMap<String, StateSpoutSpec> stateMap) {
+        return buildTopology(spoutMap, boltMap);
+    }
+
+    private static void addInputs(BoltDeclarer declarer, Map<GlobalStreamId, Grouping> inputs) {
+        for(Entry<GlobalStreamId, Grouping> entry : inputs.entrySet()) {
+            declarer.grouping(entry.getKey(), entry.getValue());
+        }
+    }
+
+    public static StormTopology buildTopology(Map<String, SpoutDetails> spoutMap, Map<String, BoltDetails> boltMap) {
+        TopologyBuilder builder = new TopologyBuilder();
+        for (Entry<String, SpoutDetails> entry : spoutMap.entrySet()) {
+            String spoutID = entry.getKey();
+            SpoutDetails spec = entry.getValue();
+            SpoutDeclarer spoutDeclarer = builder.setSpout(spoutID, spec.getSpout(), spec.getParallelism());
+            spoutDeclarer.addConfigurations(spec.getConf());
+        }
+        for (Entry<String, BoltDetails> entry : boltMap.entrySet()) {
+            String spoutID = entry.getKey();
+            BoltDetails spec = entry.getValue();
+            BoltDeclarer boltDeclarer = null;
+            if (spec.bolt instanceof IRichBolt) {
+                boltDeclarer = builder.setBolt(spoutID, (IRichBolt)spec.getBolt(), spec.getParallelism());
+            } else {
+                boltDeclarer = builder.setBolt(spoutID, (IBasicBolt)spec.getBolt(), spec.getParallelism());
+            }
+            boltDeclarer.addConfigurations(spec.getConf());
+            addInputs(boltDeclarer, spec.getInputs());
+        }
+        return builder.createTopology();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java b/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java
new file mode 100644
index 0000000..d05ae75
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.annotation;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Annotation to inform users of how much to rely on a particular package,
+ * class or method not changing over time.
+ * </ul>
+ */
+@InterfaceStability.Evolving
+public class InterfaceStability {
+    /**
+     * Can evolve while retaining compatibility for minor release boundaries.;
+     * can break compatibility only at major release (ie. at m.0).
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Stable {};
+
+    /**
+     * Evolving, but can break compatibility at minor release (i.e. m.x)
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Evolving {};
+
+    /**
+     * No guarantee is provided as to reliability or stability across any
+     * level of release granularity.
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Unstable {};
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/AtomicOutputStream.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/AtomicOutputStream.java b/storm-client/src/jvm/org/apache/storm/blobstore/AtomicOutputStream.java
new file mode 100644
index 0000000..892e25c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/AtomicOutputStream.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * An output stream where all of the data is committed on close,
+ * or can be canceled with cancel.
+ */
+public abstract class AtomicOutputStream extends OutputStream {
+    /**
+     * Cancel all of the writes associated with this stream and close it.
+     */ 
+    public abstract void cancel() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java
new file mode 100644
index 0000000..1c10c40
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java
@@ -0,0 +1,490 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import javax.security.auth.Subject;
+
+import org.apache.storm.nimbus.NimbusInfo;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.KeyNotFoundException;
+import org.apache.storm.generated.KeyAlreadyExistsException;
+import org.apache.storm.generated.ReadableBlobMeta;
+import org.apache.storm.generated.SettableBlobMeta;
+import org.apache.storm.generated.StormTopology;
+
+/**
+ * Provides a way to store blobs that can be downloaded.
+ * Blobs must be able to be uploaded and listed from Nimbus,
+ * and downloaded from the Supervisors. It is a key value based
+ * store. Key being a string and value being the blob data.
+ *
+ * ACL checking must take place against the provided subject.
+ * If the blob store does not support Security it must validate
+ * that all ACLs set are always WORLD, everything.
+ *
+ * The users can upload their blobs through the blob store command
+ * line. The command line also allows us to update and delete blobs.
+ *
+ * Modifying the replication factor only works for HdfsBlobStore
+ * as for the LocalFsBlobStore the replication is dependent on
+ * the number of Nimbodes available.
+ */
+public abstract class BlobStore implements Shutdownable {
+    private static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
+    private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
+    protected static final String BASE_BLOBS_DIR_NAME = "blobs";
+
+    /**
+     * Allows us to initialize the blob store
+     * @param conf The storm configuration
+     * @param baseDir The directory path to store the blobs
+     * @param nimbusInfo Contains the nimbus host, port and leadership information.
+     */
+    public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
+
+    /**
+     * Creates the blob.
+     * @param key Key for the blob.
+     * @param meta Metadata which contains the acls information
+     * @param who Is the subject creating the blob.
+     * @return AtomicOutputStream returns a stream into which the data
+     * can be written.
+     * @throws AuthorizationException
+     * @throws KeyAlreadyExistsException
+     */
+    public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
+
+    /**
+     * Updates the blob data.
+     * @param key Key for the blob.
+     * @param who Is the subject having the write privilege for the blob.
+     * @return AtomicOutputStream returns a stream into which the data
+     * can be written.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Gets the current version of metadata for a blob
+     * to be viewed by the user or downloaded by the supervisor.
+     * @param key Key for the blob.
+     * @param who Is the subject having the read privilege for the blob.
+     * @return AtomicOutputStream returns a stream into which the data
+     * can be written.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Sets the metadata with renewed acls for the blob.
+     * @param key Key for the blob.
+     * @param meta Metadata which contains the updated
+     * acls information.
+     * @param who Is the subject having the write privilege for the blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Deletes the blob data and metadata.
+     * @param key Key for the blob.
+     * @param who Is the subject having write privilege for the blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Gets the InputStream to read the blob details
+     * @param key Key for the blob.
+     * @param who Is the subject having the read privilege for the blob.
+     * @return InputStreamWithMeta has the additional
+     * file length and version information.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Returns an iterator with all the list of
+     * keys currently available on the blob store.
+     * @return Iterator<String>
+     */
+    public abstract Iterator<String> listKeys();
+
+    /**
+     * Gets the replication factor of the blob.
+     * @param key Key for the blob.
+     * @param who Is the subject having the read privilege for the blob.
+     * @return BlobReplication object containing the
+     * replication factor for the blob.
+     * @throws Exception
+     */
+    public abstract int getBlobReplication(String key, Subject who) throws Exception;
+
+    /**
+     * Modifies the replication factor of the blob.
+     * @param key Key for the blob.
+     * @param replication The replication factor the
+     * blob has to be set.
+     * @param who Is the subject having the update privilege for the blob
+     * @return BlobReplication object containing the
+     * updated replication factor for the blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     * @throws IOException
+     */
+    public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException;
+
+    /**
+     * Filters keys based on the KeyFilter
+     * passed as the argument.
+     * @param filter KeyFilter
+     * @param <R> Type
+     * @return Set of filtered keys
+     */
+    public <R> Set<R> filterAndListKeys(KeyFilter<R> filter) {
+        Set<R> ret = new HashSet<R>();
+        Iterator<String> keys = listKeys();
+        while (keys.hasNext()) {
+            String key = keys.next();
+            R filtered = filter.filter(key);
+            if (filtered != null) {
+                ret.add(filtered);
+            }
+        }
+        return ret;
+    }
+
+    /**
+     * Validates key checking for potentially harmful patterns
+     * @param key Key for the blob.
+     */
+    public static final void validateKey(String key) throws AuthorizationException {
+        if (StringUtils.isEmpty(key) || "..".equals(key) || ".".equals(key) || !KEY_PATTERN.matcher(key).matches()) {
+            LOG.error("'{}' does not appear to be valid {}", key, KEY_PATTERN);
+            throw new AuthorizationException(key+" does not appear to be a valid blob key");
+        }
+    }
+
+    /**
+     * Wrapper called to create the blob which contains
+     * the byte data
+     * @param key Key for the blob.
+     * @param data Byte data that needs to be uploaded.
+     * @param meta Metadata which contains the acls information
+     * @param who Is the subject creating the blob.
+     * @throws AuthorizationException
+     * @throws KeyAlreadyExistsException
+     * @throws IOException
+     */
+    public void createBlob(String key, byte [] data, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
+        AtomicOutputStream out = null;
+        try {
+            out = createBlob(key, meta, who);
+            out.write(data);
+            out.close();
+            out = null;
+        } finally {
+            if (out != null) {
+                out.cancel();
+            }
+        }
+    }
+
+    /**
+     * Wrapper called to create the blob which contains
+     * the byte data
+     * @param key Key for the blob.
+     * @param in InputStream from which the data is read to be
+     * written as a part of the blob.
+     * @param meta Metadata which contains the acls information
+     * @param who Is the subject creating the blob.
+     * @throws AuthorizationException
+     * @throws KeyAlreadyExistsException
+     * @throws IOException
+     */
+    public void createBlob(String key, InputStream in, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
+        AtomicOutputStream out = null;
+        try {
+            out = createBlob(key, meta, who);
+            byte[] buffer = new byte[2048];
+            int len = 0;
+            while ((len = in.read(buffer)) > 0) {
+                out.write(buffer, 0, len);
+            }
+            out.close();
+        } catch (AuthorizationException | IOException | RuntimeException e) {
+            if (out !=null) {
+                out.cancel();
+            }
+        } finally {
+            in.close();
+        }
+    }
+
+    /**
+     * Reads the blob from the blob store
+     * and writes it into the output stream.
+     * @param key Key for the blob.
+     * @param out Output stream
+     * @param who Is the subject having read
+     * privilege for the blob.
+     * @throws IOException
+     * @throws KeyNotFoundException
+     * @throws AuthorizationException
+     */
+    public void readBlobTo(String key, OutputStream out, Subject who) throws IOException, KeyNotFoundException, AuthorizationException {
+        InputStreamWithMeta in = getBlob(key, who);
+        if (in == null) {
+            throw new IOException("Could not find " + key);
+        }
+        byte[] buffer = new byte[2048];
+        int len = 0;
+        try{
+            while ((len = in.read(buffer)) > 0) {
+                out.write(buffer, 0, len);
+            }
+        } finally {
+            in.close();
+            out.flush();
+        }
+    }
+
+    /**
+     * Wrapper around readBlobTo which
+     * returns a ByteArray output stream.
+     * @param key  Key for the blob.
+     * @param who Is the subject having
+     * the read privilege for the blob.
+     * @return ByteArrayOutputStream
+     * @throws IOException
+     * @throws KeyNotFoundException
+     * @throws AuthorizationException
+     */
+    public byte[] readBlob(String key, Subject who) throws IOException, KeyNotFoundException, AuthorizationException {
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        readBlobTo(key, out, who);
+        byte[] bytes = out.toByteArray();
+        out.close();
+        return bytes;
+    }
+
+    /**
+     * Helper method to read a stored topology
+     * @param topoId the id of the topology to read
+     * @param who who to read it as
+     * @return the deserialized topology.
+     * @throws IOException on any error while reading the blob.
+     * @throws AuthorizationException if who is not allowed to read the blob
+     * @throws KeyNotFoundException if the blob could not be found
+     */
+    public StormTopology readTopology(String topoId, Subject who) throws KeyNotFoundException, AuthorizationException, IOException {
+        return Utils.deserialize(readBlob(ConfigUtils.masterStormCodeKey(topoId), who), StormTopology.class);
+    }
+    
+    /**
+     * Helper method to read a stored topology config
+     * @param topoId the id of the topology whose conf we are reading
+     * @param who who we are reading this as
+     * @return the deserialized config
+     * @throws KeyNotFoundException if the blob could not be found
+     * @throws AuthorizationException if who is not allowed to read the blob
+     * @throws IOException on any error while reading the blob.
+     */
+    public Map<String, Object> readTopologyConf(String topoId, Subject who) throws KeyNotFoundException, AuthorizationException, IOException {
+        return Utils.fromCompressedJsonConf(readBlob(ConfigUtils.masterStormConfKey(topoId), who));
+    }
+    
+    private static final KeyFilter<String> TO_TOPO_ID = (key) -> ConfigUtils.getIdFromBlobKey(key);
+    /**
+     * @return a set of all of the topology ids with special data stored in the
+     * blob store.
+     */
+    public Set<String> storedTopoIds() {
+        return filterAndListKeys(TO_TOPO_ID);
+    }
+    
+    /**
+     * Output stream implementation used for reading the
+     * metadata and data information.
+     */
+    protected class BlobStoreFileOutputStream extends AtomicOutputStream {
+        private BlobStoreFile part;
+        private OutputStream out;
+
+        public BlobStoreFileOutputStream(BlobStoreFile part) throws IOException {
+            this.part = part;
+            this.out = part.getOutputStream();
+        }
+
+        @Override
+        public void close() throws IOException {
+            try {
+                //close means commit
+                out.close();
+                part.commit();
+            } catch (IOException | RuntimeException e) {
+                cancel();
+                throw e;
+            }
+        }
+
+        @Override
+        public void cancel() throws IOException {
+            try {
+                out.close();
+            } finally {
+                part.cancel();
+            }
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            out.write(b);
+        }
+
+        @Override
+        public void write(byte []b) throws IOException {
+            out.write(b);
+        }
+
+        @Override
+        public void write(byte []b, int offset, int len) throws IOException {
+            out.write(b, offset, len);
+        }
+    }
+
+    /**
+     * Input stream implementation used for writing
+     * both the metadata containing the acl information
+     * and the blob data.
+     */
+    protected class BlobStoreFileInputStream extends InputStreamWithMeta {
+        private BlobStoreFile part;
+        private InputStream in;
+
+        public BlobStoreFileInputStream(BlobStoreFile part) throws IOException {
+            this.part = part;
+            this.in = part.getInputStream();
+        }
+
+        @Override
+        public long getVersion() throws IOException {
+            return part.getModTime();
+        }
+
+        @Override
+        public int read() throws IOException {
+            return in.read();
+        }
+
+        @Override
+        public int read(byte[] b, int off, int len) throws IOException {
+            return in.read(b, off, len);
+        }
+
+        @Override
+        public int read(byte[] b) throws IOException {
+            return in.read(b);
+        }
+
+        @Override
+        public int available() throws IOException {
+            return in.available();
+        }
+
+        @Override
+        public long getFileLength() throws IOException {
+            return part.getFileLength();
+        }
+
+        @Override
+        public void close() throws IOException {
+            in.close();
+        }
+    }
+
+    /**
+     * Blob store implements its own version of iterator
+     * to list the blobs
+     */
+    public static class KeyTranslationIterator implements Iterator<String> {
+        private Iterator<String> it = null;
+        private String next = null;
+        private String prefix = null;
+
+        public KeyTranslationIterator(Iterator<String> it, String prefix) throws IOException {
+            this.it = it;
+            this.prefix = prefix;
+            primeNext();
+        }
+
+        private void primeNext() {
+            next = null;
+            while (it.hasNext()) {
+                String tmp = it.next();
+                if (tmp.startsWith(prefix)) {
+                    next = tmp.substring(prefix.length());
+                    return;
+                }
+            }
+        }
+
+        @Override
+        public boolean hasNext() {
+            return next != null;
+        }
+
+        @Override
+        public String next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            String current = next;
+            primeNext();
+            return current;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException("Delete Not Supported");
+        }
+    }
+}


[19/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java b/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java
new file mode 100644
index 0000000..22a7205
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java
@@ -0,0 +1,47241 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class Nimbus {
+
+  public interface Iface {
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
+
+    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
+
+    public void killTopology(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public void activate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public void deactivate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
+
+    public void setLogConfig(String name, LogConfig config) throws org.apache.thrift.TException;
+
+    public LogConfig getLogConfig(String name) throws org.apache.thrift.TException;
+
+    /**
+     * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
+     * and if null or empty, the debug flag will apply to the entire topology.
+     * 
+     * The 'samplingPercentage' will limit loggging to a percentage of generated tuples.
+     * 
+     * 
+     * @param name
+     * @param component
+     * @param enable
+     * @param samplingPercentage
+     */
+    public void debug(String name, String component, boolean enable, double samplingPercentage) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest) throws org.apache.thrift.TException;
+
+    public List<ProfileRequest> getComponentPendingProfileActions(String id, String component_id, ProfileAction action) throws org.apache.thrift.TException;
+
+    public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
+
+    public String beginCreateBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException, org.apache.thrift.TException;
+
+    public String beginUpdateBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public void uploadBlobChunk(String session, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void finishBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void cancelBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException;
+
+    public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public BeginDownloadResult beginBlobDownload(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public ByteBuffer downloadBlobChunk(String session) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public ListBlobsResult listBlobs(String session) throws org.apache.thrift.TException;
+
+    public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public void createStateInZookeeper(String key) throws org.apache.thrift.TException;
+
+    public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException;
+
+    public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void finishFileUpload(String location) throws AuthorizationException, org.apache.thrift.TException;
+
+    public String beginFileDownload(String file) throws AuthorizationException, org.apache.thrift.TException;
+
+    public ByteBuffer downloadChunk(String id) throws AuthorizationException, org.apache.thrift.TException;
+
+    public String getNimbusConf() throws AuthorizationException, org.apache.thrift.TException;
+
+    public ClusterSummary getClusterInfo() throws AuthorizationException, org.apache.thrift.TException;
+
+    public NimbusSummary getLeader() throws AuthorizationException, org.apache.thrift.TException;
+
+    public boolean isTopologyNameAllowed(String name) throws AuthorizationException, org.apache.thrift.TException;
+
+    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public TopologyInfo getTopologyInfoWithOpts(String id, GetInfoOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public TopologyPageInfo getTopologyPageInfo(String id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public SupervisorPageInfo getSupervisorPageInfo(String id, String host, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public ComponentPageInfo getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    /**
+     * Returns the compiled topology that contains ackers and metrics consumsers. Compare {@link #getUserTopology(String id)}.
+     * 
+     * @param id
+     */
+    public StormTopology getTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    /**
+     * Returns the user specified topology as submitted originally. Compare {@link #getTopology(String id)}.
+     * 
+     * @param id
+     */
+    public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public TopologyHistoryInfo getTopologyHistory(String user) throws AuthorizationException, org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void killTopology(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void activate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void deactivate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void setLogConfig(String name, LogConfig config, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getLogConfig(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void debug(String name, String component, boolean enable, double samplingPercentage, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getComponentPendingProfileActions(String id, String component_id, ProfileAction action, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void beginCreateBlob(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void beginUpdateBlob(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void uploadBlobChunk(String session, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void finishBlobUpload(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void cancelBlobUpload(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getBlobMeta(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void setBlobMeta(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void beginBlobDownload(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void downloadBlobChunk(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void deleteBlob(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void listBlobs(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getBlobReplication(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void updateBlobReplication(String key, int replication, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void createStateInZookeeper(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void finishFileUpload(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void beginFileDownload(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void downloadChunk(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getNimbusConf(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getClusterInfo(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getLeader(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void isTopologyNameAllowed(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTopologyInfo(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTopologyInfoWithOpts(String id, GetInfoOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTopologyPageInfo(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getSupervisorPageInfo(String id, String host, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getUserTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTopologyHistory(String user, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_submitTopology(name, uploadedJarLocation, jsonConf, topology);
+      recv_submitTopology();
+    }
+
+    public void send_submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws org.apache.thrift.TException
+    {
+      submitTopology_args args = new submitTopology_args();
+      args.set_name(name);
+      args.set_uploadedJarLocation(uploadedJarLocation);
+      args.set_jsonConf(jsonConf);
+      args.set_topology(topology);
+      sendBase("submitTopology", args);
+    }
+
+    public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      submitTopology_result result = new submitTopology_result();
+      receiveBase(result, "submitTopology");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.ite != null) {
+        throw result.ite;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options);
+      recv_submitTopologyWithOpts();
+    }
+
+    public void send_submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws org.apache.thrift.TException
+    {
+      submitTopologyWithOpts_args args = new submitTopologyWithOpts_args();
+      args.set_name(name);
+      args.set_uploadedJarLocation(uploadedJarLocation);
+      args.set_jsonConf(jsonConf);
+      args.set_topology(topology);
+      args.set_options(options);
+      sendBase("submitTopologyWithOpts", args);
+    }
+
+    public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      submitTopologyWithOpts_result result = new submitTopologyWithOpts_result();
+      receiveBase(result, "submitTopologyWithOpts");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.ite != null) {
+        throw result.ite;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void killTopology(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_killTopology(name);
+      recv_killTopology();
+    }
+
+    public void send_killTopology(String name) throws org.apache.thrift.TException
+    {
+      killTopology_args args = new killTopology_args();
+      args.set_name(name);
+      sendBase("killTopology", args);
+    }
+
+    public void recv_killTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      killTopology_result result = new killTopology_result();
+      receiveBase(result, "killTopology");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_killTopologyWithOpts(name, options);
+      recv_killTopologyWithOpts();
+    }
+
+    public void send_killTopologyWithOpts(String name, KillOptions options) throws org.apache.thrift.TException
+    {
+      killTopologyWithOpts_args args = new killTopologyWithOpts_args();
+      args.set_name(name);
+      args.set_options(options);
+      sendBase("killTopologyWithOpts", args);
+    }
+
+    public void recv_killTopologyWithOpts() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      killTopologyWithOpts_result result = new killTopologyWithOpts_result();
+      receiveBase(result, "killTopologyWithOpts");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void activate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_activate(name);
+      recv_activate();
+    }
+
+    public void send_activate(String name) throws org.apache.thrift.TException
+    {
+      activate_args args = new activate_args();
+      args.set_name(name);
+      sendBase("activate", args);
+    }
+
+    public void recv_activate() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      activate_result result = new activate_result();
+      receiveBase(result, "activate");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void deactivate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_deactivate(name);
+      recv_deactivate();
+    }
+
+    public void send_deactivate(String name) throws org.apache.thrift.TException
+    {
+      deactivate_args args = new deactivate_args();
+      args.set_name(name);
+      sendBase("deactivate", args);
+    }
+
+    public void recv_deactivate() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      deactivate_result result = new deactivate_result();
+      receiveBase(result, "deactivate");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_rebalance(name, options);
+      recv_rebalance();
+    }
+
+    public void send_rebalance(String name, RebalanceOptions options) throws org.apache.thrift.TException
+    {
+      rebalance_args args = new rebalance_args();
+      args.set_name(name);
+      args.set_options(options);
+      sendBase("rebalance", args);
+    }
+
+    public void recv_rebalance() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      rebalance_result result = new rebalance_result();
+      receiveBase(result, "rebalance");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.ite != null) {
+        throw result.ite;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void setLogConfig(String name, LogConfig config) throws org.apache.thrift.TException
+    {
+      send_setLogConfig(name, config);
+      recv_setLogConfig();
+    }
+
+    public void send_setLogConfig(String name, LogConfig config) throws org.apache.thrift.TException
+    {
+      setLogConfig_args args = new setLogConfig_args();
+      args.set_name(name);
+      args.set_config(config);
+      sendBase("setLogConfig", args);
+    }
+
+    public void recv_setLogConfig() throws org.apache.thrift.TException
+    {
+      setLogConfig_result result = new setLogConfig_result();
+      receiveBase(result, "setLogConfig");
+      return;
+    }
+
+    public LogConfig getLogConfig(String name) throws org.apache.thrift.TException
+    {
+      send_getLogConfig(name);
+      return recv_getLogConfig();
+    }
+
+    public void send_getLogConfig(String name) throws org.apache.thrift.TException
+    {
+      getLogConfig_args args = new getLogConfig_args();
+      args.set_name(name);
+      sendBase("getLogConfig", args);
+    }
+
+    public LogConfig recv_getLogConfig() throws org.apache.thrift.TException
+    {
+      getLogConfig_result result = new getLogConfig_result();
+      receiveBase(result, "getLogConfig");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getLogConfig failed: unknown result");
+    }
+
+    public void debug(String name, String component, boolean enable, double samplingPercentage) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_debug(name, component, enable, samplingPercentage);
+      recv_debug();
+    }
+
+    public void send_debug(String name, String component, boolean enable, double samplingPercentage) throws org.apache.thrift.TException
+    {
+      debug_args args = new debug_args();
+      args.set_name(name);
+      args.set_component(component);
+      args.set_enable(enable);
+      args.set_samplingPercentage(samplingPercentage);
+      sendBase("debug", args);
+    }
+
+    public void recv_debug() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      debug_result result = new debug_result();
+      receiveBase(result, "debug");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest) throws org.apache.thrift.TException
+    {
+      send_setWorkerProfiler(id, profileRequest);
+      recv_setWorkerProfiler();
+    }
+
+    public void send_setWorkerProfiler(String id, ProfileRequest profileRequest) throws org.apache.thrift.TException
+    {
+      setWorkerProfiler_args args = new setWorkerProfiler_args();
+      args.set_id(id);
+      args.set_profileRequest(profileRequest);
+      sendBase("setWorkerProfiler", args);
+    }
+
+    public void recv_setWorkerProfiler() throws org.apache.thrift.TException
+    {
+      setWorkerProfiler_result result = new setWorkerProfiler_result();
+      receiveBase(result, "setWorkerProfiler");
+      return;
+    }
+
+    public List<ProfileRequest> getComponentPendingProfileActions(String id, String component_id, ProfileAction action) throws org.apache.thrift.TException
+    {
+      send_getComponentPendingProfileActions(id, component_id, action);
+      return recv_getComponentPendingProfileActions();
+    }
+
+    public void send_getComponentPendingProfileActions(String id, String component_id, ProfileAction action) throws org.apache.thrift.TException
+    {
+      getComponentPendingProfileActions_args args = new getComponentPendingProfileActions_args();
+      args.set_id(id);
+      args.set_component_id(component_id);
+      args.set_action(action);
+      sendBase("getComponentPendingProfileActions", args);
+    }
+
+    public List<ProfileRequest> recv_getComponentPendingProfileActions() throws org.apache.thrift.TException
+    {
+      getComponentPendingProfileActions_result result = new getComponentPendingProfileActions_result();
+      receiveBase(result, "getComponentPendingProfileActions");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getComponentPendingProfileActions failed: unknown result");
+    }
+
+    public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_uploadNewCredentials(name, creds);
+      recv_uploadNewCredentials();
+    }
+
+    public void send_uploadNewCredentials(String name, Credentials creds) throws org.apache.thrift.TException
+    {
+      uploadNewCredentials_args args = new uploadNewCredentials_args();
+      args.set_name(name);
+      args.set_creds(creds);
+      sendBase("uploadNewCredentials", args);
+    }
+
+    public void recv_uploadNewCredentials() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      uploadNewCredentials_result result = new uploadNewCredentials_result();
+      receiveBase(result, "uploadNewCredentials");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.ite != null) {
+        throw result.ite;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public String beginCreateBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException, org.apache.thrift.TException
+    {
+      send_beginCreateBlob(key, meta);
+      return recv_beginCreateBlob();
+    }
+
+    public void send_beginCreateBlob(String key, SettableBlobMeta meta) throws org.apache.thrift.TException
+    {
+      beginCreateBlob_args args = new beginCreateBlob_args();
+      args.set_key(key);
+      args.set_meta(meta);
+      sendBase("beginCreateBlob", args);
+    }
+
+    public String recv_beginCreateBlob() throws AuthorizationException, KeyAlreadyExistsException, org.apache.thrift.TException
+    {
+      beginCreateBlob_result result = new beginCreateBlob_result();
+      receiveBase(result, "beginCreateBlob");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.kae != null) {
+        throw result.kae;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginCreateBlob failed: unknown result");
+    }
+
+    public String beginUpdateBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_beginUpdateBlob(key);
+      return recv_beginUpdateBlob();
+    }
+
+    public void send_beginUpdateBlob(String key) throws org.apache.thrift.TException
+    {
+      beginUpdateBlob_args args = new beginUpdateBlob_args();
+      args.set_key(key);
+      sendBase("beginUpdateBlob", args);
+    }
+
+    public String recv_beginUpdateBlob() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      beginUpdateBlob_result result = new beginUpdateBlob_result();
+      receiveBase(result, "beginUpdateBlob");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginUpdateBlob failed: unknown result");
+    }
+
+    public void uploadBlobChunk(String session, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_uploadBlobChunk(session, chunk);
+      recv_uploadBlobChunk();
+    }
+
+    public void send_uploadBlobChunk(String session, ByteBuffer chunk) throws org.apache.thrift.TException
+    {
+      uploadBlobChunk_args args = new uploadBlobChunk_args();
+      args.set_session(session);
+      args.set_chunk(chunk);
+      sendBase("uploadBlobChunk", args);
+    }
+
+    public void recv_uploadBlobChunk() throws AuthorizationException, org.apache.thrift.TException
+    {
+      uploadBlobChunk_result result = new uploadBlobChunk_result();
+      receiveBase(result, "uploadBlobChunk");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void finishBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_finishBlobUpload(session);
+      recv_finishBlobUpload();
+    }
+
+    public void send_finishBlobUpload(String session) throws org.apache.thrift.TException
+    {
+      finishBlobUpload_args args = new finishBlobUpload_args();
+      args.set_session(session);
+      sendBase("finishBlobUpload", args);
+    }
+
+    public void recv_finishBlobUpload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      finishBlobUpload_result result = new finishBlobUpload_result();
+      receiveBase(result, "finishBlobUpload");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void cancelBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_cancelBlobUpload(session);
+      recv_cancelBlobUpload();
+    }
+
+    public void send_cancelBlobUpload(String session) throws org.apache.thrift.TException
+    {
+      cancelBlobUpload_args args = new cancelBlobUpload_args();
+      args.set_session(session);
+      sendBase("cancelBlobUpload", args);
+    }
+
+    public void recv_cancelBlobUpload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      cancelBlobUpload_result result = new cancelBlobUpload_result();
+      receiveBase(result, "cancelBlobUpload");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_getBlobMeta(key);
+      return recv_getBlobMeta();
+    }
+
+    public void send_getBlobMeta(String key) throws org.apache.thrift.TException
+    {
+      getBlobMeta_args args = new getBlobMeta_args();
+      args.set_key(key);
+      sendBase("getBlobMeta", args);
+    }
+
+    public ReadableBlobMeta recv_getBlobMeta() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      getBlobMeta_result result = new getBlobMeta_result();
+      receiveBase(result, "getBlobMeta");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getBlobMeta failed: unknown result");
+    }
+
+    public void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_setBlobMeta(key, meta);
+      recv_setBlobMeta();
+    }
+
+    public void send_setBlobMeta(String key, SettableBlobMeta meta) throws org.apache.thrift.TException
+    {
+      setBlobMeta_args args = new setBlobMeta_args();
+      args.set_key(key);
+      args.set_meta(meta);
+      sendBase("setBlobMeta", args);
+    }
+
+    public void recv_setBlobMeta() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      setBlobMeta_result result = new setBlobMeta_result();
+      receiveBase(result, "setBlobMeta");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      return;
+    }
+
+    public BeginDownloadResult beginBlobDownload(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_beginBlobDownload(key);
+      return recv_beginBlobDownload();
+    }
+
+    public void send_beginBlobDownload(String key) throws org.apache.thrift.TException
+    {
+      beginBlobDownload_args args = new beginBlobDownload_args();
+      args.set_key(key);
+      sendBase("beginBlobDownload", args);
+    }
+
+    public BeginDownloadResult recv_beginBlobDownload() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      beginBlobDownload_result result = new beginBlobDownload_result();
+      receiveBase(result, "beginBlobDownload");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginBlobDownload failed: unknown result");
+    }
+
+    public ByteBuffer downloadBlobChunk(String session) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_downloadBlobChunk(session);
+      return recv_downloadBlobChunk();
+    }
+
+    public void send_downloadBlobChunk(String session) throws org.apache.thrift.TException
+    {
+      downloadBlobChunk_args args = new downloadBlobChunk_args();
+      args.set_session(session);
+      sendBase("downloadBlobChunk", args);
+    }
+
+    public ByteBuffer recv_downloadBlobChunk() throws AuthorizationException, org.apache.thrift.TException
+    {
+      downloadBlobChunk_result result = new downloadBlobChunk_result();
+      receiveBase(result, "downloadBlobChunk");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "downloadBlobChunk failed: unknown result");
+    }
+
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_deleteBlob(key);
+      recv_deleteBlob();
+    }
+
+    public void send_deleteBlob(String key) throws org.apache.thrift.TException
+    {
+      deleteBlob_args args = new deleteBlob_args();
+      args.set_key(key);
+      sendBase("deleteBlob", args);
+    }
+
+    public void recv_deleteBlob() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      deleteBlob_result result = new deleteBlob_result();
+      receiveBase(result, "deleteBlob");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      return;
+    }
+
+    public ListBlobsResult listBlobs(String session) throws org.apache.thrift.TException
+    {
+      send_listBlobs(session);
+      return recv_listBlobs();
+    }
+
+    public void send_listBlobs(String session) throws org.apache.thrift.TException
+    {
+      listBlobs_args args = new listBlobs_args();
+      args.set_session(session);
+      sendBase("listBlobs", args);
+    }
+
+    public ListBlobsResult recv_listBlobs() throws org.apache.thrift.TException
+    {
+      listBlobs_result result = new listBlobs_result();
+      receiveBase(result, "listBlobs");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "listBlobs failed: unknown result");
+    }
+
+    public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_getBlobReplication(key);
+      return recv_getBlobReplication();
+    }
+
+    public void send_getBlobReplication(String key) throws org.apache.thrift.TException
+    {
+      getBlobReplication_args args = new getBlobReplication_args();
+      args.set_key(key);
+      sendBase("getBlobReplication", args);
+    }
+
+    public int recv_getBlobReplication() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      getBlobReplication_result result = new getBlobReplication_result();
+      receiveBase(result, "getBlobReplication");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getBlobReplication failed: unknown result");
+    }
+
+    public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_updateBlobReplication(key, replication);
+      return recv_updateBlobReplication();
+    }
+
+    public void send_updateBlobReplication(String key, int replication) throws org.apache.thrift.TException
+    {
+      updateBlobReplication_args args = new updateBlobReplication_args();
+      args.set_key(key);
+      args.set_replication(replication);
+      sendBase("updateBlobReplication", args);
+    }
+
+    public int recv_updateBlobReplication() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      updateBlobReplication_result result = new updateBlobReplication_result();
+      receiveBase(result, "updateBlobReplication");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "updateBlobReplication failed: unknown result");
+    }
+
+    public void createStateInZookeeper(String key) throws org.apache.thrift.TException
+    {
+      send_createStateInZookeeper(key);
+      recv_createStateInZookeeper();
+    }
+
+    public void send_createStateInZookeeper(String key) throws org.apache.thrift.TException
+    {
+      createStateInZookeeper_args args = new createStateInZookeeper_args();
+      args.set_key(key);
+      sendBase("createStateInZookeeper", args);
+    }
+
+    public void recv_createStateInZookeeper() throws org.apache.thrift.TException
+    {
+      createStateInZookeeper_result result = new createStateInZookeeper_result();
+      receiveBase(result, "createStateInZookeeper");
+      return;
+    }
+
+    public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_beginFileUpload();
+      return recv_beginFileUpload();
+    }
+
+    public void send_beginFileUpload() throws org.apache.thrift.TException
+    {
+      beginFileUpload_args args = new beginFileUpload_args();
+      sendBase("beginFileUpload", args);
+    }
+
+    public String recv_beginFileUpload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      beginFileUpload_result result = new beginFileUpload_result();
+      receiveBase(result, "beginFileUpload");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result");
+    }
+
+    public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_uploadChunk(location, chunk);
+      recv_uploadChunk();
+    }
+
+    public void send_uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException
+    {
+      uploadChunk_args args = new uploadChunk_args();
+      args.set_location(location);
+      args.set_chunk(chunk);
+      sendBase("uploadChunk", args);
+    }
+
+    public void recv_uploadChunk() throws AuthorizationException, org.apache.thrift.TException
+    {
+      uploadChunk_result result = new uploadChunk_result();
+      receiveBase(result, "uploadChunk");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void finishFileUpload(String location) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_finishFileUpload(location);
+      recv_finishFileUpload();
+    }
+
+    public void send_finishFileUpload(String location) throws org.apache.thrift.TException
+    {
+      finishFileUpload_args args = new finishFileUpload_args();
+      args.set_location(location);
+      sendBase("finishFileUpload", args);
+    }
+
+    public void recv_finishFileUpload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      finishFileUpload_result result = new finishFileUpload_result();
+      receiveBase(result, "finishFileUpload");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public String beginFileDownload(String file) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_beginFileDownload(file);
+      return recv_beginFileDownload();
+    }
+
+    public void send_beginFileDownload(String file) throws org.apache.thrift.TException
+    {
+      beginFileDownload_args args = new beginFileDownload_args();
+      args.set_file(file);
+      sendBase("beginFileDownload", args);
+    }
+
+    public String recv_beginFileDownload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      beginFileDownload_result result = new beginFileDownload_result();
+      receiveBase(result, "beginFileDownload");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result");
+    }
+
+    public ByteBuffer downloadChunk(String id) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_downloadChunk(id);
+      return recv_downloadChunk();
+    }
+
+    public void send_downloadChunk(String id) throws org.apache.thrift.TException
+    {
+      downloadChunk_args args = new downloadChunk_args();
+      args.set_id(id);
+      sendBase("downloadChunk", args);
+    }
+
+    public ByteBuffer recv_downloadChunk() throws AuthorizationException, org.apache.thrift.TException
+    {
+      downloadChunk_result result = new downloadChunk_result();
+      receiveBase(result, "downloadChunk");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result");
+    }
+
+    public String getNimbusConf() throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_getNimbusConf();
+      return recv_getNimbusConf();
+    }
+
+    public void send_getNimbusConf() throws org.apache.thrift.TException
+    {
+      getNimbusConf_args args = new getNimbusConf_args();
+      sendBase("getNimbusConf", args);
+    }
+
+    public String recv_getNimbusConf() throws AuthorizationException, org.apache.thrift.TException
+    {
+      getNimbusConf_result result = new getNimbusConf_result();
+      receiveBase(result, "getNimbusConf");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result");
+    }
+
+    public ClusterSummary getClusterInfo() throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_getClusterInfo();
+      return recv_getClusterInfo();
+    }
+
+    public void send_getClusterInfo() throws org.apache.thrift.TException
+    {
+      getClusterInfo_args args = new getClusterInfo_args();
+      sendBase("getClusterInfo", args);
+    }
+
+    public ClusterSummary recv_getClusterInfo() throws AuthorizationException, org.apache.thrift.TException
+    {
+      getClusterInfo_result result = new getClusterInfo_result();
+      receiveBase(result, "getClusterInfo");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result");
+    }
+
+    public NimbusSummary getLeader() throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_getLeader();
+      return recv_getLeader();
+    }
+
+    public void send_getLeader() throws org.apache.thrift.TException
+    {
+      getLeader_args args = new getLeader_args();
+      sendBase("getLeader", args);
+    }
+
+    public NimbusSummary recv_getLeader() throws AuthorizationException, org.apache.thrift.TException
+    {
+      getLeader_result result = new getLeader_result();
+      receiveBase(result, "getLeader");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getLeader failed: unknown result");
+    }
+
+    public boolean isTopologyNameAllowed(String name) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_isTopologyNameAllowed(name);
+      return recv_isTopologyNameAllowed();
+    }
+
+    public void send_isTopologyNameAllowed(String name) throws org.apache.thrift.TException
+    {
+      isTopologyNameAllowed_args args = new isTopologyNameAllowed_args();
+      args.set_name(name);
+      sendBase("isTopologyNameAllowed", args);
+    }
+
+    public boolean recv_isTopologyNameAllowed() throws AuthorizationException, org.apache.thrift.TException
+    {
+      isTopologyNameAllowed_result result = new isTopologyNameAllowed_result();
+      receiveBase(result, "isTopologyNameAllowed");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "isTopologyNameAllowed failed: unknown result");
+    }
+
+    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopologyInfo(id);
+      return recv_getTopologyInfo();
+    }
+
+    public void send_getTopologyInfo(String id) throws org.apache.thrift.TException
+    {
+      getTopologyInfo_args args = new getTopologyInfo_args();
+      args.set_id(id);
+      sendBase("getTopologyInfo", args);
+    }
+
+    public TopologyInfo recv_getTopologyInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getTopologyInfo_result result = new getTopologyInfo_result();
+      receiveBase(result, "getTopologyInfo");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result");
+    }
+
+    public TopologyInfo getTopologyInfoWithOpts(String id, GetInfoOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopologyInfoWithOpts(id, options);
+      return recv_getTopologyInfoWithOpts();
+    }
+
+    public void send_getTopologyInfoWithOpts(String id, GetInfoOptions options) throws org.apache.thrift.TException
+    {
+      getTopologyInfoWithOpts_args args = new getTopologyInfoWithOpts_args();
+      args.set_id(id);
+      args.set_options(options);
+      sendBase("getTopologyInfoWithOpts", args);
+    }
+
+    public TopologyInfo recv_getTopologyInfoWithOpts() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getTopologyInfoWithOpts_result result = new getTopologyInfoWithOpts_result();
+      receiveBase(result, "getTopologyInfoWithOpts");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result");
+    }
+
+    public TopologyPageInfo getTopologyPageInfo(String id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopologyPageInfo(id, window, is_include_sys);
+      return recv_getTopologyPageInfo();
+    }
+
+    public void send_getTopologyPageInfo(String id, String window, boolean is_include_sys) throws org.apache.thrift.TException
+    {
+      getTopologyPageInfo_args args = new getTopologyPageInfo_args();
+      args.set_id(id);
+      args.set_window(window);
+      args.set_is_include_sys(is_include_sys);
+      sendBase("getTopologyPageInfo", args);
+    }
+
+    public TopologyPageInfo recv_getTopologyPageInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getTopologyPageInfo_result result = new getTopologyPageInfo_result();
+      receiveBase(result, "getTopologyPageInfo");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyPageInfo failed: unknown result");
+    }
+
+    public SupervisorPageInfo getSupervisorPageInfo(String id, String host, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getSupervisorPageInfo(id, host, is_include_sys);
+      return recv_getSupervisorPageInfo();
+    }
+
+    public void send_getSupervisorPageInfo(String id, String host, boolean is_include_sys) throws org.apache.thrift.TException
+    {
+      getSupervisorPageInfo_args args = new getSupervisorPageInfo_args();
+      args.set_id(id);
+      args.set_host(host);
+      args.set_is_include_sys(is_include_sys);
+      sendBase("getSupervisorPageInfo", args);
+    }
+
+    public SupervisorPageInfo recv_getSupervisorPageInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getSupervisorPageInfo_result result = new getSupervisorPageInfo_result();
+      receiveBase(result, "getSupervisorPageInfo");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getSupervisorPageInfo failed: unknown result");
+    }
+
+    public ComponentPageInfo getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getComponentPageInfo(topology_id, component_id, window, is_include_sys);
+      return recv_getComponentPageInfo();
+    }
+
+    public void send_getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws org.apache.thrift.TException
+    {
+      getComponentPageInfo_args args = new getComponentPageInfo_args();
+      args.set_topology_id(topology_id);
+      args.set_component_id(component_id);
+      args.set_window(window);
+      args.set_is_include_sys(is_include_sys);
+      sendBase("getComponentPageInfo", args);
+    }
+
+    public ComponentPageInfo recv_getComponentPageInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getComponentPageInfo_result result = new getComponentPageInfo_result();
+      receiveBase(result, "getComponentPageInfo");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getComponentPageInfo failed: unknown result");
+    }
+
+    public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopologyConf(id);
+      return recv_getTopologyConf();
+    }
+
+    public void send_getTopologyConf(String id) throws org.apache.thrift.TException
+    {
+      getTopologyConf_args args = new getTopologyConf_args();
+      args.set_id(id);
+      sendBase("getTopologyConf", args);
+    }
+
+    public String recv_getTopologyConf() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getTopologyConf_result result = new getTopologyConf_result();
+      receiveBase(result, "getTopologyConf");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result");
+    }
+
+    public StormTopology getTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopology(id);
+      return recv_getTopology();
+    }
+
+    public void send_getTopology(String id) throws org.apache.thrift.TException
+    {
+      getTopology_args args = new getTopology_args();
+      args.set_id(id);
+      sendBase("getTopology", args);
+    }
+
+    public StormTopology recv_getTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getTopology_result result = new getTopology_result();
+      receiveBase(result, "getTopology");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopology failed: unknown result");
+    }
+
+    public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getUserTopology(id);
+      return recv_getUserTopology();
+    }
+
+    public void send_getUserTopology(String id) throws org.apache.thrift.TException
+    {
+      getUserTopology_args args = new getUserTopology_args();
+      args.set_id(id);
+      sendBase("getUserTopology", args);
+    }
+
+    public StormTopology recv_getUserTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getUserTopology_result result = new getUserTopology_result();
+      receiveBase(result, "getUserTopology");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result");
+    }
+
+    public TopologyHistoryInfo getTopologyHistory(String user) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopologyHistory(user);
+      return recv_getTopologyHistory();
+    }
+
+    public void send_getTopologyHistory(String user) throws org.apache.thrift.TException
+    {
+      getTopologyHistory_args args = new getTopologyHistory_args();
+      args.set_user(user);
+      sendBase("getTopologyHistory", args);
+    }
+
+    public TopologyHistoryInfo recv_getTopologyHistory() throws AuthorizationException, org.apache.thrift.TException
+    {
+      getTopologyHistory_result result = new getTopologyHistory_result();
+      receiveBase(result, "getTopologyHistory");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyHistory failed: unknown result");
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      submitTopology_call method_call = new submitTopology_call(name, uploadedJarLocation, jsonConf, topology, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class submitTopology_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private String uploadedJarLocation;
+      private String jsonConf;
+      private StormTopology topology;
+      public submitTopology_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.uploadedJarLocation = uploadedJarLocation;
+        this.jsonConf = jsonConf;
+        this.topology = topology;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("submitTopology", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        submitTopology_args args = new submitTopology_args();
+        args.set_name(name);
+        args.set_uploadedJarLocation(uploadedJarLocation);
+        args.set_jsonConf(jsonConf);
+        args.set_topology(topology);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_submitTopology();
+      }
+    }
+
+    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      submitTopologyWithOpts_call method_call = new submitTopologyWithOpts_call(name, uploadedJarLocation, jsonConf, topology, options, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class submitTopologyWithOpts_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private String uploadedJarLocation;
+      private String jsonConf;
+      private StormTopology topology;
+      private SubmitOptions options;
+      public submitTopologyWithOpts_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.uploadedJarLocation = uploadedJarLocation;
+        this.jsonConf = jsonConf;
+        this.topology = topology;
+        this.options = options;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("submitTopologyWithOpts", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        submitTopologyWithOpts_args args = new submitTopologyWithOpts_args();
+        args.set_name(name);
+        args.set_uploadedJarLocation(uploadedJarLocation);
+        args.set_jsonConf(jsonConf);
+        args.set_topology(topology);
+        args.set_options(options);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_submitTopologyWithOpts();
+      }
+    }
+
+    public void killTopology(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      killTopology_call method_call = new killTopology_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class killTopology_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      public killTopology_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("killTopology", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        killTopology_args args = new killTopology_args();
+        args.set_name(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_killTopology();
+      }
+    }
+
+    public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      killTopologyWithOpts_call method_call = new killTopologyWithOpts_call(name, options, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class killTopologyWithOpts_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private KillOptions options;
+      public killTopologyWithOpts_call(String name, KillOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.options = options;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("killTopologyWithOpts", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        killTopologyWithOpts_args args = new killTopologyWithOpts_args();
+        args.set_name(name);
+        args.set_options(options);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_killTopologyWithOpts();
+      }
+    }
+
+    public void activate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      activate_call method_call = new activate_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class activate_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      public activate_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("activate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        activate_args args = new activate_args();
+        args.set_name(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_activate();
+      }
+    }
+
+    public void deactivate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      deactivate_call method_call = new deactivate_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class deactivate_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      public deactivate_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("deactivate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        deactivate_args args = new deactivate_args();
+        args.set_name(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_deactivate();
+      }
+    }
+
+    public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      rebalance_call method_call = new rebalance_call(name, options, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class rebalance_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private RebalanceOptions options;
+      public rebalance_call(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.options = options;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("rebalance", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        rebalance_args args = new rebalance_args();
+        args.set_name(name);
+        args.set_options(options);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_rebalance();
+      }
+    }
+
+    public void setLogConfig(String name, LogConfig config, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      setLogConfig_call method_call = new setLogConfig_call(name, config, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class setLogConfig_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private LogConfig config;
+      public setLogConfig_call(String name, LogConfig config, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.config = config;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setLogConfig", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        setLogConfig_args args = new setLogConfig_args();
+        args.set_name(name);
+        args.set_config(config);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_setLogConfig();
+      }
+    }
+
+    public void getLogConfig(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getLogConfig_call method_call = new getLogConfig_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getLogConfig_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      public getLogConfig_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getLogConfig", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getLogConfig_args args = new getLogConfig_args();
+        args.set_name(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public LogConfig getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getLogConfig();
+      }
+    }
+
+    public void debug(String name, String component, boolean enable, double samplingPercentage, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      debug_call method_call = new debug_call(name, component, enable, samplingPercentage, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class debug_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private String component;
+      private boolean enable;
+      private double samplingPercentage;
+      public debug_call(String name, String component, boolean enable, double samplingPercentage, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.component = component;
+        this.enable = enable;
+        this.samplingPercentage = samplingPercentage;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("debug", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        debug_args args = new debug_args();
+        args.set_name(name);
+        args.set_component(component);
+        args.set_enable(enable);
+        args.set_samplingPercentage(samplingPercentage);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_debug();
+      }
+    }
+
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      setWorkerProfiler_call method_call = new setWorkerProfiler_call(id, profileRequest, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class setWorkerProfiler_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      private ProfileRequest profileRequest;
+      public setWorkerProfiler_call(String id, ProfileRequest profileRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.profileRequest = profileRequest;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setWorkerProfiler", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        setWorkerProfiler_args args = new setWorkerProfiler_args();
+        args.set_id(id);
+        args.set_profileRequest(profileRequest);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_setWorkerProfiler();
+      }
+    }
+
+    public void getComponentPendingProfileActions(String id, String component_id, ProfileAction action, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getComponentPendingProfileActions_call method_call = new getComponentPendingProfileActions_call(id, component_id, action, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getComponentPendingProfileActions_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      private String component_id;
+      private ProfileAction action;
+      public getComponentPendingProfileActions_call(String id, String component_id, ProfileAction action, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.component_id = component_id;
+        this.action = action;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getComponentPendingProfileActions", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getComponentPendingProfileActions_args args = new getComponentPendingProfileActions_args();
+        args.set_id(id);
+        args.set_component_id(component_id);
+        args.set_action(action);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<ProfileRequest> getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransp

<TRUNCATED>

[09/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyInfo.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyInfo.java
new file mode 100644
index 0000000..fa95be7
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyInfo.java
@@ -0,0 +1,2144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
+
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.MAP, (short)6);
+  private static final org.apache.thrift.protocol.TField COMPONENT_DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("component_debug", org.apache.thrift.protocol.TType.MAP, (short)7);
+  private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
+  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)514);
+  private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)515);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologyInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologyInfoTupleSchemeFactory());
+  }
+
+  private String id; // required
+  private String name; // required
+  private int uptime_secs; // required
+  private List<ExecutorSummary> executors; // required
+  private String status; // required
+  private Map<String,List<ErrorInfo>> errors; // required
+  private Map<String,DebugOptions> component_debug; // optional
+  private String sched_status; // optional
+  private String owner; // optional
+  private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ID((short)1, "id"),
+    NAME((short)2, "name"),
+    UPTIME_SECS((short)3, "uptime_secs"),
+    EXECUTORS((short)4, "executors"),
+    STATUS((short)5, "status"),
+    ERRORS((short)6, "errors"),
+    COMPONENT_DEBUG((short)7, "component_debug"),
+    SCHED_STATUS((short)513, "sched_status"),
+    OWNER((short)514, "owner"),
+    REPLICATION_COUNT((short)515, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // NAME
+          return NAME;
+        case 3: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 4: // EXECUTORS
+          return EXECUTORS;
+        case 5: // STATUS
+          return STATUS;
+        case 6: // ERRORS
+          return ERRORS;
+        case 7: // COMPONENT_DEBUG
+          return COMPONENT_DEBUG;
+        case 513: // SCHED_STATUS
+          return SCHED_STATUS;
+        case 514: // OWNER
+          return OWNER;
+        case 515: // REPLICATION_COUNT
+          return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __UPTIME_SECS_ISSET_ID = 0;
+  private static final int __REPLICATION_COUNT_ISSET_ID = 1;
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 2;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 3;
+  private static final int __REQUESTED_CPU_ISSET_ID = 4;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 7;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("executors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSummary.class))));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+                new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class)))));
+    tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class))));
+    tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap);
+  }
+
+  public TopologyInfo() {
+  }
+
+  public TopologyInfo(
+    String id,
+    String name,
+    int uptime_secs,
+    List<ExecutorSummary> executors,
+    String status,
+    Map<String,List<ErrorInfo>> errors)
+  {
+    this();
+    this.id = id;
+    this.name = name;
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+    this.executors = executors;
+    this.status = status;
+    this.errors = errors;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyInfo(TopologyInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_executors()) {
+      List<ExecutorSummary> __this__executors = new ArrayList<ExecutorSummary>(other.executors.size());
+      for (ExecutorSummary other_element : other.executors) {
+        __this__executors.add(new ExecutorSummary(other_element));
+      }
+      this.executors = __this__executors;
+    }
+    if (other.is_set_status()) {
+      this.status = other.status;
+    }
+    if (other.is_set_errors()) {
+      Map<String,List<ErrorInfo>> __this__errors = new HashMap<String,List<ErrorInfo>>(other.errors.size());
+      for (Map.Entry<String, List<ErrorInfo>> other_element : other.errors.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        List<ErrorInfo> other_element_value = other_element.getValue();
+
+        String __this__errors_copy_key = other_element_key;
+
+        List<ErrorInfo> __this__errors_copy_value = new ArrayList<ErrorInfo>(other_element_value.size());
+        for (ErrorInfo other_element_value_element : other_element_value) {
+          __this__errors_copy_value.add(new ErrorInfo(other_element_value_element));
+        }
+
+        __this__errors.put(__this__errors_copy_key, __this__errors_copy_value);
+      }
+      this.errors = __this__errors;
+    }
+    if (other.is_set_component_debug()) {
+      Map<String,DebugOptions> __this__component_debug = new HashMap<String,DebugOptions>(other.component_debug.size());
+      for (Map.Entry<String, DebugOptions> other_element : other.component_debug.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        DebugOptions other_element_value = other_element.getValue();
+
+        String __this__component_debug_copy_key = other_element_key;
+
+        DebugOptions __this__component_debug_copy_value = new DebugOptions(other_element_value);
+
+        __this__component_debug.put(__this__component_debug_copy_key, __this__component_debug_copy_value);
+      }
+      this.component_debug = __this__component_debug;
+    }
+    if (other.is_set_sched_status()) {
+      this.sched_status = other.sched_status;
+    }
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
+    this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
+  }
+
+  public TopologyInfo deepCopy() {
+    return new TopologyInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.id = null;
+    this.name = null;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    this.executors = null;
+    this.status = null;
+    this.errors = null;
+    this.component_debug = null;
+    this.sched_status = null;
+    this.owner = null;
+    set_replication_count_isSet(false);
+    this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_executors_size() {
+    return (this.executors == null) ? 0 : this.executors.size();
+  }
+
+  public java.util.Iterator<ExecutorSummary> get_executors_iterator() {
+    return (this.executors == null) ? null : this.executors.iterator();
+  }
+
+  public void add_to_executors(ExecutorSummary elem) {
+    if (this.executors == null) {
+      this.executors = new ArrayList<ExecutorSummary>();
+    }
+    this.executors.add(elem);
+  }
+
+  public List<ExecutorSummary> get_executors() {
+    return this.executors;
+  }
+
+  public void set_executors(List<ExecutorSummary> executors) {
+    this.executors = executors;
+  }
+
+  public void unset_executors() {
+    this.executors = null;
+  }
+
+  /** Returns true if field executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executors() {
+    return this.executors != null;
+  }
+
+  public void set_executors_isSet(boolean value) {
+    if (!value) {
+      this.executors = null;
+    }
+  }
+
+  public String get_status() {
+    return this.status;
+  }
+
+  public void set_status(String status) {
+    this.status = status;
+  }
+
+  public void unset_status() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_status() {
+    return this.status != null;
+  }
+
+  public void set_status_isSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public int get_errors_size() {
+    return (this.errors == null) ? 0 : this.errors.size();
+  }
+
+  public void put_to_errors(String key, List<ErrorInfo> val) {
+    if (this.errors == null) {
+      this.errors = new HashMap<String,List<ErrorInfo>>();
+    }
+    this.errors.put(key, val);
+  }
+
+  public Map<String,List<ErrorInfo>> get_errors() {
+    return this.errors;
+  }
+
+  public void set_errors(Map<String,List<ErrorInfo>> errors) {
+    this.errors = errors;
+  }
+
+  public void unset_errors() {
+    this.errors = null;
+  }
+
+  /** Returns true if field errors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_errors() {
+    return this.errors != null;
+  }
+
+  public void set_errors_isSet(boolean value) {
+    if (!value) {
+      this.errors = null;
+    }
+  }
+
+  public int get_component_debug_size() {
+    return (this.component_debug == null) ? 0 : this.component_debug.size();
+  }
+
+  public void put_to_component_debug(String key, DebugOptions val) {
+    if (this.component_debug == null) {
+      this.component_debug = new HashMap<String,DebugOptions>();
+    }
+    this.component_debug.put(key, val);
+  }
+
+  public Map<String,DebugOptions> get_component_debug() {
+    return this.component_debug;
+  }
+
+  public void set_component_debug(Map<String,DebugOptions> component_debug) {
+    this.component_debug = component_debug;
+  }
+
+  public void unset_component_debug() {
+    this.component_debug = null;
+  }
+
+  /** Returns true if field component_debug is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_debug() {
+    return this.component_debug != null;
+  }
+
+  public void set_component_debug_isSet(boolean value) {
+    if (!value) {
+      this.component_debug = null;
+    }
+  }
+
+  public String get_sched_status() {
+    return this.sched_status;
+  }
+
+  public void set_sched_status(String sched_status) {
+    this.sched_status = sched_status;
+  }
+
+  public void unset_sched_status() {
+    this.sched_status = null;
+  }
+
+  /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sched_status() {
+    return this.sched_status != null;
+  }
+
+  public void set_sched_status_isSet(boolean value) {
+    if (!value) {
+      this.sched_status = null;
+    }
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = null;
+  }
+
+  /** Returns true if field owner is set (has been assigned a value) and false otherwise */
+  public boolean is_set_owner() {
+    return this.owner != null;
+  }
+
+  public void set_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
+  public int get_replication_count() {
+    return this.replication_count;
+  }
+
+  public void set_replication_count(int replication_count) {
+    this.replication_count = replication_count;
+    set_replication_count_isSet(true);
+  }
+
+  public void unset_replication_count() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_count() {
+    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  public void set_replication_count_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
+  }
+
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case EXECUTORS:
+      if (value == null) {
+        unset_executors();
+      } else {
+        set_executors((List<ExecutorSummary>)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unset_status();
+      } else {
+        set_status((String)value);
+      }
+      break;
+
+    case ERRORS:
+      if (value == null) {
+        unset_errors();
+      } else {
+        set_errors((Map<String,List<ErrorInfo>>)value);
+      }
+      break;
+
+    case COMPONENT_DEBUG:
+      if (value == null) {
+        unset_component_debug();
+      } else {
+        set_component_debug((Map<String,DebugOptions>)value);
+      }
+      break;
+
+    case SCHED_STATUS:
+      if (value == null) {
+        unset_sched_status();
+      } else {
+        set_sched_status((String)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
+    case REPLICATION_COUNT:
+      if (value == null) {
+        unset_replication_count();
+      } else {
+        set_replication_count((Integer)value);
+      }
+      break;
+
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case NAME:
+      return get_name();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case EXECUTORS:
+      return get_executors();
+
+    case STATUS:
+      return get_status();
+
+    case ERRORS:
+      return get_errors();
+
+    case COMPONENT_DEBUG:
+      return get_component_debug();
+
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case OWNER:
+      return get_owner();
+
+    case REPLICATION_COUNT:
+      return get_replication_count();
+
+    case REQUESTED_MEMONHEAP:
+      return get_requested_memonheap();
+
+    case REQUESTED_MEMOFFHEAP:
+      return get_requested_memoffheap();
+
+    case REQUESTED_CPU:
+      return get_requested_cpu();
+
+    case ASSIGNED_MEMONHEAP:
+      return get_assigned_memonheap();
+
+    case ASSIGNED_MEMOFFHEAP:
+      return get_assigned_memoffheap();
+
+    case ASSIGNED_CPU:
+      return get_assigned_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case NAME:
+      return is_set_name();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case EXECUTORS:
+      return is_set_executors();
+    case STATUS:
+      return is_set_status();
+    case ERRORS:
+      return is_set_errors();
+    case COMPONENT_DEBUG:
+      return is_set_component_debug();
+    case SCHED_STATUS:
+      return is_set_sched_status();
+    case OWNER:
+      return is_set_owner();
+    case REPLICATION_COUNT:
+      return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyInfo)
+      return this.equals((TopologyInfo)that);
+    return false;
+  }
+
+  public boolean equals(TopologyInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_executors = true && this.is_set_executors();
+    boolean that_present_executors = true && that.is_set_executors();
+    if (this_present_executors || that_present_executors) {
+      if (!(this_present_executors && that_present_executors))
+        return false;
+      if (!this.executors.equals(that.executors))
+        return false;
+    }
+
+    boolean this_present_status = true && this.is_set_status();
+    boolean that_present_status = true && that.is_set_status();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    boolean this_present_errors = true && this.is_set_errors();
+    boolean that_present_errors = true && that.is_set_errors();
+    if (this_present_errors || that_present_errors) {
+      if (!(this_present_errors && that_present_errors))
+        return false;
+      if (!this.errors.equals(that.errors))
+        return false;
+    }
+
+    boolean this_present_component_debug = true && this.is_set_component_debug();
+    boolean that_present_component_debug = true && that.is_set_component_debug();
+    if (this_present_component_debug || that_present_component_debug) {
+      if (!(this_present_component_debug && that_present_component_debug))
+        return false;
+      if (!this.component_debug.equals(that.component_debug))
+        return false;
+    }
+
+    boolean this_present_sched_status = true && this.is_set_sched_status();
+    boolean that_present_sched_status = true && that.is_set_sched_status();
+    if (this_present_sched_status || that_present_sched_status) {
+      if (!(this_present_sched_status && that_present_sched_status))
+        return false;
+      if (!this.sched_status.equals(that.sched_status))
+        return false;
+    }
+
+    boolean this_present_owner = true && this.is_set_owner();
+    boolean that_present_owner = true && that.is_set_owner();
+    if (this_present_owner || that_present_owner) {
+      if (!(this_present_owner && that_present_owner))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
+    boolean this_present_replication_count = true && this.is_set_replication_count();
+    boolean that_present_replication_count = true && that.is_set_replication_count();
+    if (this_present_replication_count || that_present_replication_count) {
+      if (!(this_present_replication_count && that_present_replication_count))
+        return false;
+      if (this.replication_count != that.replication_count)
+        return false;
+    }
+
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_id = true && (is_set_id());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_uptime_secs = true;
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_executors = true && (is_set_executors());
+    list.add(present_executors);
+    if (present_executors)
+      list.add(executors);
+
+    boolean present_status = true && (is_set_status());
+    list.add(present_status);
+    if (present_status)
+      list.add(status);
+
+    boolean present_errors = true && (is_set_errors());
+    list.add(present_errors);
+    if (present_errors)
+      list.add(errors);
+
+    boolean present_component_debug = true && (is_set_component_debug());
+    list.add(present_component_debug);
+    if (present_component_debug)
+      list.add(component_debug);
+
+    boolean present_sched_status = true && (is_set_sched_status());
+    list.add(present_sched_status);
+    if (present_sched_status)
+      list.add(sched_status);
+
+    boolean present_owner = true && (is_set_owner());
+    list.add(present_owner);
+    if (present_owner)
+      list.add(owner);
+
+    boolean present_replication_count = true && (is_set_replication_count());
+    list.add(present_replication_count);
+    if (present_replication_count)
+      list.add(replication_count);
+
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologyInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executors()).compareTo(other.is_set_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executors, other.executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_errors()).compareTo(other.is_set_errors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_errors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errors, other.errors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_debug()).compareTo(other.is_set_component_debug());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_debug()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_debug, other.component_debug);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(other.is_set_sched_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sched_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, other.sched_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(other.is_set_replication_count());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_count()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, other.replication_count);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyInfo(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executors:");
+    if (this.executors == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executors);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("status:");
+    if (this.status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.status);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("errors:");
+    if (this.errors == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.errors);
+    }
+    first = false;
+    if (is_set_component_debug()) {
+      if (!first) sb.append(", ");
+      sb.append("component_debug:");
+      if (this.component_debug == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_debug);
+      }
+      first = false;
+    }
+    if (is_set_sched_status()) {
+      if (!first) sb.append(", ");
+      sb.append("sched_status:");
+      if (this.sched_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sched_status);
+      }
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
+    if (is_set_replication_count()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_count:");
+      sb.append(this.replication_count);
+      first = false;
+    }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executors()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executors' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_status()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_errors()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'errors' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologyInfoStandardSchemeFactory implements SchemeFactory {
+    public TopologyInfoStandardScheme getScheme() {
+      return new TopologyInfoStandardScheme();
+    }
+  }
+
+  private static class TopologyInfoStandardScheme extends StandardScheme<TopologyInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.id = iprot.readString();
+              struct.set_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list332 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorSummary>(_list332.size);
+                ExecutorSummary _elem333;
+                for (int _i334 = 0; _i334 < _list332.size; ++_i334)
+                {
+                  _elem333 = new ExecutorSummary();
+                  _elem333.read(iprot);
+                  struct.executors.add(_elem333);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.status = iprot.readString();
+              struct.set_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // ERRORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map335 = iprot.readMapBegin();
+                struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map335.size);
+                String _key336;
+                List<ErrorInfo> _val337;
+                for (int _i338 = 0; _i338 < _map335.size; ++_i338)
+                {
+                  _key336 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TList _list339 = iprot.readListBegin();
+                    _val337 = new ArrayList<ErrorInfo>(_list339.size);
+                    ErrorInfo _elem340;
+                    for (int _i341 = 0; _i341 < _list339.size; ++_i341)
+                    {
+                      _elem340 = new ErrorInfo();
+                      _elem340.read(iprot);
+                      _val337.add(_elem340);
+                    }
+                    iprot.readListEnd();
+                  }
+                  struct.errors.put(_key336, _val337);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_errors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // COMPONENT_DEBUG
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map342 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map342.size);
+                String _key343;
+                DebugOptions _val344;
+                for (int _i345 = 0; _i345 < _map342.size; ++_i345)
+                {
+                  _key343 = iprot.readString();
+                  _val344 = new DebugOptions();
+                  _val344.read(iprot);
+                  struct.component_debug.put(_key343, _val344);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_component_debug_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 513: // SCHED_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.sched_status = iprot.readString();
+              struct.set_sched_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 514: // OWNER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.owner = iprot.readString();
+              struct.set_owner_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 515: // REPLICATION_COUNT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.replication_count = iprot.readI32();
+              struct.set_replication_count_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(struct.id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeFieldEnd();
+      if (struct.executors != null) {
+        oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
+          for (ExecutorSummary _iter346 : struct.executors)
+          {
+            _iter346.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.status != null) {
+        oprot.writeFieldBegin(STATUS_FIELD_DESC);
+        oprot.writeString(struct.status);
+        oprot.writeFieldEnd();
+      }
+      if (struct.errors != null) {
+        oprot.writeFieldBegin(ERRORS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.errors.size()));
+          for (Map.Entry<String, List<ErrorInfo>> _iter347 : struct.errors.entrySet())
+          {
+            oprot.writeString(_iter347.getKey());
+            {
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter347.getValue().size()));
+              for (ErrorInfo _iter348 : _iter347.getValue())
+              {
+                _iter348.write(oprot);
+              }
+              oprot.writeListEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.component_debug != null) {
+        if (struct.is_set_component_debug()) {
+          oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.component_debug.size()));
+            for (Map.Entry<String, DebugOptions> _iter349 : struct.component_debug.entrySet())
+            {
+              oprot.writeString(_iter349.getKey());
+              _iter349.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.sched_status != null) {
+        if (struct.is_set_sched_status()) {
+          oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
+          oprot.writeString(struct.sched_status);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.owner != null) {
+        if (struct.is_set_owner()) {
+          oprot.writeFieldBegin(OWNER_FIELD_DESC);
+          oprot.writeString(struct.owner);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_replication_count()) {
+        oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC);
+        oprot.writeI32(struct.replication_count);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TopologyInfoTupleSchemeFactory implements SchemeFactory {
+    public TopologyInfoTupleScheme getScheme() {
+      return new TopologyInfoTupleScheme();
+    }
+  }
+
+  private static class TopologyInfoTupleScheme extends TupleScheme<TopologyInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TopologyInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.id);
+      oprot.writeString(struct.name);
+      oprot.writeI32(struct.uptime_secs);
+      {
+        oprot.writeI32(struct.executors.size());
+        for (ExecutorSummary _iter350 : struct.executors)
+        {
+          _iter350.write(oprot);
+        }
+      }
+      oprot.writeString(struct.status);
+      {
+        oprot.writeI32(struct.errors.size());
+        for (Map.Entry<String, List<ErrorInfo>> _iter351 : struct.errors.entrySet())
+        {
+          oprot.writeString(_iter351.getKey());
+          {
+            oprot.writeI32(_iter351.getValue().size());
+            for (ErrorInfo _iter352 : _iter351.getValue())
+            {
+              _iter352.write(oprot);
+            }
+          }
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_component_debug()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_sched_status()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_owner()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_replication_count()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(8);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(9);
+      }
+      oprot.writeBitSet(optionals, 10);
+      if (struct.is_set_component_debug()) {
+        {
+          oprot.writeI32(struct.component_debug.size());
+          for (Map.Entry<String, DebugOptions> _iter353 : struct.component_debug.entrySet())
+          {
+            oprot.writeString(_iter353.getKey());
+            _iter353.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_sched_status()) {
+        oprot.writeString(struct.sched_status);
+      }
+      if (struct.is_set_owner()) {
+        oprot.writeString(struct.owner);
+      }
+      if (struct.is_set_replication_count()) {
+        oprot.writeI32(struct.replication_count);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TopologyInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.id = iprot.readString();
+      struct.set_id_isSet(true);
+      struct.name = iprot.readString();
+      struct.set_name_isSet(true);
+      struct.uptime_secs = iprot.readI32();
+      struct.set_uptime_secs_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list354 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorSummary>(_list354.size);
+        ExecutorSummary _elem355;
+        for (int _i356 = 0; _i356 < _list354.size; ++_i356)
+        {
+          _elem355 = new ExecutorSummary();
+          _elem355.read(iprot);
+          struct.executors.add(_elem355);
+        }
+      }
+      struct.set_executors_isSet(true);
+      struct.status = iprot.readString();
+      struct.set_status_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map357 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+        struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map357.size);
+        String _key358;
+        List<ErrorInfo> _val359;
+        for (int _i360 = 0; _i360 < _map357.size; ++_i360)
+        {
+          _key358 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TList _list361 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val359 = new ArrayList<ErrorInfo>(_list361.size);
+            ErrorInfo _elem362;
+            for (int _i363 = 0; _i363 < _list361.size; ++_i363)
+            {
+              _elem362 = new ErrorInfo();
+              _elem362.read(iprot);
+              _val359.add(_elem362);
+            }
+          }
+          struct.errors.put(_key358, _val359);
+        }
+      }
+      struct.set_errors_isSet(true);
+      BitSet incoming = iprot.readBitSet(10);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map364 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.component_debug = new HashMap<String,DebugOptions>(2*_map364.size);
+          String _key365;
+          DebugOptions _val366;
+          for (int _i367 = 0; _i367 < _map364.size; ++_i367)
+          {
+            _key365 = iprot.readString();
+            _val366 = new DebugOptions();
+            _val366.read(iprot);
+            struct.component_debug.put(_key365, _val366);
+          }
+        }
+        struct.set_component_debug_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.sched_status = iprot.readString();
+        struct.set_sched_status_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.owner = iprot.readString();
+        struct.set_owner_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.replication_count = iprot.readI32();
+        struct.set_replication_count_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologyInitialStatus.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyInitialStatus.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyInitialStatus.java
new file mode 100644
index 0000000..5cf2894
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyInitialStatus.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum TopologyInitialStatus implements org.apache.thrift.TEnum {
+  ACTIVE(1),
+  INACTIVE(2);
+
+  private final int value;
+
+  private TopologyInitialStatus(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TopologyInitialStatus findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return ACTIVE;
+      case 2:
+        return INACTIVE;
+      default:
+        return null;
+    }
+  }
+}


[11/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SupervisorSummary.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SupervisorSummary.java b/storm-client/src/jvm/org/apache/storm/generated/SupervisorSummary.java
new file mode 100644
index 0000000..bd7d7a4
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorSummary.java
@@ -0,0 +1,1265 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSummary, SupervisorSummary._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorSummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorSummary");
+
+  private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField NUM_USED_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_used_workers", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_id", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField TOTAL_RESOURCES_FIELD_DESC = new org.apache.thrift.protocol.TField("total_resources", org.apache.thrift.protocol.TType.MAP, (short)7);
+  private static final org.apache.thrift.protocol.TField USED_MEM_FIELD_DESC = new org.apache.thrift.protocol.TField("used_mem", org.apache.thrift.protocol.TType.DOUBLE, (short)8);
+  private static final org.apache.thrift.protocol.TField USED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("used_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)9);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SupervisorSummaryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SupervisorSummaryTupleSchemeFactory());
+  }
+
+  private String host; // required
+  private int uptime_secs; // required
+  private int num_workers; // required
+  private int num_used_workers; // required
+  private String supervisor_id; // required
+  private String version; // optional
+  private Map<String,Double> total_resources; // optional
+  private double used_mem; // optional
+  private double used_cpu; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    HOST((short)1, "host"),
+    UPTIME_SECS((short)2, "uptime_secs"),
+    NUM_WORKERS((short)3, "num_workers"),
+    NUM_USED_WORKERS((short)4, "num_used_workers"),
+    SUPERVISOR_ID((short)5, "supervisor_id"),
+    VERSION((short)6, "version"),
+    TOTAL_RESOURCES((short)7, "total_resources"),
+    USED_MEM((short)8, "used_mem"),
+    USED_CPU((short)9, "used_cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // HOST
+          return HOST;
+        case 2: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 3: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 4: // NUM_USED_WORKERS
+          return NUM_USED_WORKERS;
+        case 5: // SUPERVISOR_ID
+          return SUPERVISOR_ID;
+        case 6: // VERSION
+          return VERSION;
+        case 7: // TOTAL_RESOURCES
+          return TOTAL_RESOURCES;
+        case 8: // USED_MEM
+          return USED_MEM;
+        case 9: // USED_CPU
+          return USED_CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __UPTIME_SECS_ISSET_ID = 0;
+  private static final int __NUM_WORKERS_ISSET_ID = 1;
+  private static final int __NUM_USED_WORKERS_ISSET_ID = 2;
+  private static final int __USED_MEM_ISSET_ID = 3;
+  private static final int __USED_CPU_ISSET_ID = 4;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.VERSION,_Fields.TOTAL_RESOURCES,_Fields.USED_MEM,_Fields.USED_CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_USED_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_used_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift.meta_data.FieldMetaData("supervisor_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOTAL_RESOURCES, new org.apache.thrift.meta_data.FieldMetaData("total_resources", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
+    tmpMap.put(_Fields.USED_MEM, new org.apache.thrift.meta_data.FieldMetaData("used_mem", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.USED_CPU, new org.apache.thrift.meta_data.FieldMetaData("used_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap);
+  }
+
+  public SupervisorSummary() {
+    this.version = "VERSION_NOT_PROVIDED";
+
+  }
+
+  public SupervisorSummary(
+    String host,
+    int uptime_secs,
+    int num_workers,
+    int num_used_workers,
+    String supervisor_id)
+  {
+    this();
+    this.host = host;
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+    this.num_used_workers = num_used_workers;
+    set_num_used_workers_isSet(true);
+    this.supervisor_id = supervisor_id;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SupervisorSummary(SupervisorSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.uptime_secs = other.uptime_secs;
+    this.num_workers = other.num_workers;
+    this.num_used_workers = other.num_used_workers;
+    if (other.is_set_supervisor_id()) {
+      this.supervisor_id = other.supervisor_id;
+    }
+    if (other.is_set_version()) {
+      this.version = other.version;
+    }
+    if (other.is_set_total_resources()) {
+      Map<String,Double> __this__total_resources = new HashMap<String,Double>(other.total_resources);
+      this.total_resources = __this__total_resources;
+    }
+    this.used_mem = other.used_mem;
+    this.used_cpu = other.used_cpu;
+  }
+
+  public SupervisorSummary deepCopy() {
+    return new SupervisorSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.host = null;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    set_num_workers_isSet(false);
+    this.num_workers = 0;
+    set_num_used_workers_isSet(false);
+    this.num_used_workers = 0;
+    this.supervisor_id = null;
+    this.version = "VERSION_NOT_PROVIDED";
+
+    this.total_resources = null;
+    set_used_mem_isSet(false);
+    this.used_mem = 0.0;
+    set_used_cpu_isSet(false);
+    this.used_cpu = 0.0;
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_num_used_workers() {
+    return this.num_used_workers;
+  }
+
+  public void set_num_used_workers(int num_used_workers) {
+    this.num_used_workers = num_used_workers;
+    set_num_used_workers_isSet(true);
+  }
+
+  public void unset_num_used_workers() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_used_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_used_workers() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_used_workers_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID, value);
+  }
+
+  public String get_supervisor_id() {
+    return this.supervisor_id;
+  }
+
+  public void set_supervisor_id(String supervisor_id) {
+    this.supervisor_id = supervisor_id;
+  }
+
+  public void unset_supervisor_id() {
+    this.supervisor_id = null;
+  }
+
+  /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_supervisor_id() {
+    return this.supervisor_id != null;
+  }
+
+  public void set_supervisor_id_isSet(boolean value) {
+    if (!value) {
+      this.supervisor_id = null;
+    }
+  }
+
+  public String get_version() {
+    return this.version;
+  }
+
+  public void set_version(String version) {
+    this.version = version;
+  }
+
+  public void unset_version() {
+    this.version = null;
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return this.version != null;
+  }
+
+  public void set_version_isSet(boolean value) {
+    if (!value) {
+      this.version = null;
+    }
+  }
+
+  public int get_total_resources_size() {
+    return (this.total_resources == null) ? 0 : this.total_resources.size();
+  }
+
+  public void put_to_total_resources(String key, double val) {
+    if (this.total_resources == null) {
+      this.total_resources = new HashMap<String,Double>();
+    }
+    this.total_resources.put(key, val);
+  }
+
+  public Map<String,Double> get_total_resources() {
+    return this.total_resources;
+  }
+
+  public void set_total_resources(Map<String,Double> total_resources) {
+    this.total_resources = total_resources;
+  }
+
+  public void unset_total_resources() {
+    this.total_resources = null;
+  }
+
+  /** Returns true if field total_resources is set (has been assigned a value) and false otherwise */
+  public boolean is_set_total_resources() {
+    return this.total_resources != null;
+  }
+
+  public void set_total_resources_isSet(boolean value) {
+    if (!value) {
+      this.total_resources = null;
+    }
+  }
+
+  public double get_used_mem() {
+    return this.used_mem;
+  }
+
+  public void set_used_mem(double used_mem) {
+    this.used_mem = used_mem;
+    set_used_mem_isSet(true);
+  }
+
+  public void unset_used_mem() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USED_MEM_ISSET_ID);
+  }
+
+  /** Returns true if field used_mem is set (has been assigned a value) and false otherwise */
+  public boolean is_set_used_mem() {
+    return EncodingUtils.testBit(__isset_bitfield, __USED_MEM_ISSET_ID);
+  }
+
+  public void set_used_mem_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USED_MEM_ISSET_ID, value);
+  }
+
+  public double get_used_cpu() {
+    return this.used_cpu;
+  }
+
+  public void set_used_cpu(double used_cpu) {
+    this.used_cpu = used_cpu;
+    set_used_cpu_isSet(true);
+  }
+
+  public void unset_used_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field used_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_used_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __USED_CPU_ISSET_ID);
+  }
+
+  public void set_used_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USED_CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case NUM_USED_WORKERS:
+      if (value == null) {
+        unset_num_used_workers();
+      } else {
+        set_num_used_workers((Integer)value);
+      }
+      break;
+
+    case SUPERVISOR_ID:
+      if (value == null) {
+        unset_supervisor_id();
+      } else {
+        set_supervisor_id((String)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((String)value);
+      }
+      break;
+
+    case TOTAL_RESOURCES:
+      if (value == null) {
+        unset_total_resources();
+      } else {
+        set_total_resources((Map<String,Double>)value);
+      }
+      break;
+
+    case USED_MEM:
+      if (value == null) {
+        unset_used_mem();
+      } else {
+        set_used_mem((Double)value);
+      }
+      break;
+
+    case USED_CPU:
+      if (value == null) {
+        unset_used_cpu();
+      } else {
+        set_used_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case HOST:
+      return get_host();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case NUM_WORKERS:
+      return get_num_workers();
+
+    case NUM_USED_WORKERS:
+      return get_num_used_workers();
+
+    case SUPERVISOR_ID:
+      return get_supervisor_id();
+
+    case VERSION:
+      return get_version();
+
+    case TOTAL_RESOURCES:
+      return get_total_resources();
+
+    case USED_MEM:
+      return get_used_mem();
+
+    case USED_CPU:
+      return get_used_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case HOST:
+      return is_set_host();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case NUM_USED_WORKERS:
+      return is_set_num_used_workers();
+    case SUPERVISOR_ID:
+      return is_set_supervisor_id();
+    case VERSION:
+      return is_set_version();
+    case TOTAL_RESOURCES:
+      return is_set_total_resources();
+    case USED_MEM:
+      return is_set_used_mem();
+    case USED_CPU:
+      return is_set_used_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SupervisorSummary)
+      return this.equals((SupervisorSummary)that);
+    return false;
+  }
+
+  public boolean equals(SupervisorSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_num_workers = true;
+    boolean that_present_num_workers = true;
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_num_used_workers = true;
+    boolean that_present_num_used_workers = true;
+    if (this_present_num_used_workers || that_present_num_used_workers) {
+      if (!(this_present_num_used_workers && that_present_num_used_workers))
+        return false;
+      if (this.num_used_workers != that.num_used_workers)
+        return false;
+    }
+
+    boolean this_present_supervisor_id = true && this.is_set_supervisor_id();
+    boolean that_present_supervisor_id = true && that.is_set_supervisor_id();
+    if (this_present_supervisor_id || that_present_supervisor_id) {
+      if (!(this_present_supervisor_id && that_present_supervisor_id))
+        return false;
+      if (!this.supervisor_id.equals(that.supervisor_id))
+        return false;
+    }
+
+    boolean this_present_version = true && this.is_set_version();
+    boolean that_present_version = true && that.is_set_version();
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (!this.version.equals(that.version))
+        return false;
+    }
+
+    boolean this_present_total_resources = true && this.is_set_total_resources();
+    boolean that_present_total_resources = true && that.is_set_total_resources();
+    if (this_present_total_resources || that_present_total_resources) {
+      if (!(this_present_total_resources && that_present_total_resources))
+        return false;
+      if (!this.total_resources.equals(that.total_resources))
+        return false;
+    }
+
+    boolean this_present_used_mem = true && this.is_set_used_mem();
+    boolean that_present_used_mem = true && that.is_set_used_mem();
+    if (this_present_used_mem || that_present_used_mem) {
+      if (!(this_present_used_mem && that_present_used_mem))
+        return false;
+      if (this.used_mem != that.used_mem)
+        return false;
+    }
+
+    boolean this_present_used_cpu = true && this.is_set_used_cpu();
+    boolean that_present_used_cpu = true && that.is_set_used_cpu();
+    if (this_present_used_cpu || that_present_used_cpu) {
+      if (!(this_present_used_cpu && that_present_used_cpu))
+        return false;
+      if (this.used_cpu != that.used_cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_host = true && (is_set_host());
+    list.add(present_host);
+    if (present_host)
+      list.add(host);
+
+    boolean present_uptime_secs = true;
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_num_workers = true;
+    list.add(present_num_workers);
+    if (present_num_workers)
+      list.add(num_workers);
+
+    boolean present_num_used_workers = true;
+    list.add(present_num_used_workers);
+    if (present_num_used_workers)
+      list.add(num_used_workers);
+
+    boolean present_supervisor_id = true && (is_set_supervisor_id());
+    list.add(present_supervisor_id);
+    if (present_supervisor_id)
+      list.add(supervisor_id);
+
+    boolean present_version = true && (is_set_version());
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    boolean present_total_resources = true && (is_set_total_resources());
+    list.add(present_total_resources);
+    if (present_total_resources)
+      list.add(total_resources);
+
+    boolean present_used_mem = true && (is_set_used_mem());
+    list.add(present_used_mem);
+    if (present_used_mem)
+      list.add(used_mem);
+
+    boolean present_used_cpu = true && (is_set_used_cpu());
+    list.add(present_used_cpu);
+    if (present_used_cpu)
+      list.add(used_cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SupervisorSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_used_workers()).compareTo(other.is_set_num_used_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_used_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_used_workers, other.num_used_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(other.is_set_supervisor_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_supervisor_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_id, other.supervisor_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_total_resources()).compareTo(other.is_set_total_resources());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_total_resources()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.total_resources, other.total_resources);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_used_mem()).compareTo(other.is_set_used_mem());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_used_mem()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_mem, other.used_mem);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_used_cpu()).compareTo(other.is_set_used_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_used_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_cpu, other.used_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SupervisorSummary(");
+    boolean first = true;
+
+    sb.append("host:");
+    if (this.host == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.host);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_workers:");
+    sb.append(this.num_workers);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_used_workers:");
+    sb.append(this.num_used_workers);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("supervisor_id:");
+    if (this.supervisor_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.supervisor_id);
+    }
+    first = false;
+    if (is_set_version()) {
+      if (!first) sb.append(", ");
+      sb.append("version:");
+      if (this.version == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.version);
+      }
+      first = false;
+    }
+    if (is_set_total_resources()) {
+      if (!first) sb.append(", ");
+      sb.append("total_resources:");
+      if (this.total_resources == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.total_resources);
+      }
+      first = false;
+    }
+    if (is_set_used_mem()) {
+      if (!first) sb.append(", ");
+      sb.append("used_mem:");
+      sb.append(this.used_mem);
+      first = false;
+    }
+    if (is_set_used_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("used_cpu:");
+      sb.append(this.used_cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_host()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_workers()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_used_workers()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_used_workers' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_supervisor_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor_id' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SupervisorSummaryStandardSchemeFactory implements SchemeFactory {
+    public SupervisorSummaryStandardScheme getScheme() {
+      return new SupervisorSummaryStandardScheme();
+    }
+  }
+
+  private static class SupervisorSummaryStandardScheme extends StandardScheme<SupervisorSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.host = iprot.readString();
+              struct.set_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NUM_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_workers = iprot.readI32();
+              struct.set_num_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // NUM_USED_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_used_workers = iprot.readI32();
+              struct.set_num_used_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // SUPERVISOR_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.supervisor_id = iprot.readString();
+              struct.set_supervisor_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.version = iprot.readString();
+              struct.set_version_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // TOTAL_RESOURCES
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map98 = iprot.readMapBegin();
+                struct.total_resources = new HashMap<String,Double>(2*_map98.size);
+                String _key99;
+                double _val100;
+                for (int _i101 = 0; _i101 < _map98.size; ++_i101)
+                {
+                  _key99 = iprot.readString();
+                  _val100 = iprot.readDouble();
+                  struct.total_resources.put(_key99, _val100);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_total_resources_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // USED_MEM
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.used_mem = iprot.readDouble();
+              struct.set_used_mem_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // USED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.used_cpu = iprot.readDouble();
+              struct.set_used_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.host != null) {
+        oprot.writeFieldBegin(HOST_FIELD_DESC);
+        oprot.writeString(struct.host);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+      oprot.writeI32(struct.num_workers);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(NUM_USED_WORKERS_FIELD_DESC);
+      oprot.writeI32(struct.num_used_workers);
+      oprot.writeFieldEnd();
+      if (struct.supervisor_id != null) {
+        oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC);
+        oprot.writeString(struct.supervisor_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.version != null) {
+        if (struct.is_set_version()) {
+          oprot.writeFieldBegin(VERSION_FIELD_DESC);
+          oprot.writeString(struct.version);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.total_resources != null) {
+        if (struct.is_set_total_resources()) {
+          oprot.writeFieldBegin(TOTAL_RESOURCES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.total_resources.size()));
+            for (Map.Entry<String, Double> _iter102 : struct.total_resources.entrySet())
+            {
+              oprot.writeString(_iter102.getKey());
+              oprot.writeDouble(_iter102.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_used_mem()) {
+        oprot.writeFieldBegin(USED_MEM_FIELD_DESC);
+        oprot.writeDouble(struct.used_mem);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_used_cpu()) {
+        oprot.writeFieldBegin(USED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.used_cpu);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SupervisorSummaryTupleSchemeFactory implements SchemeFactory {
+    public SupervisorSummaryTupleScheme getScheme() {
+      return new SupervisorSummaryTupleScheme();
+    }
+  }
+
+  private static class SupervisorSummaryTupleScheme extends TupleScheme<SupervisorSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.host);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeI32(struct.num_workers);
+      oprot.writeI32(struct.num_used_workers);
+      oprot.writeString(struct.supervisor_id);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_version()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_total_resources()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_used_mem()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_used_cpu()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_version()) {
+        oprot.writeString(struct.version);
+      }
+      if (struct.is_set_total_resources()) {
+        {
+          oprot.writeI32(struct.total_resources.size());
+          for (Map.Entry<String, Double> _iter103 : struct.total_resources.entrySet())
+          {
+            oprot.writeString(_iter103.getKey());
+            oprot.writeDouble(_iter103.getValue());
+          }
+        }
+      }
+      if (struct.is_set_used_mem()) {
+        oprot.writeDouble(struct.used_mem);
+      }
+      if (struct.is_set_used_cpu()) {
+        oprot.writeDouble(struct.used_cpu);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.host = iprot.readString();
+      struct.set_host_isSet(true);
+      struct.uptime_secs = iprot.readI32();
+      struct.set_uptime_secs_isSet(true);
+      struct.num_workers = iprot.readI32();
+      struct.set_num_workers_isSet(true);
+      struct.num_used_workers = iprot.readI32();
+      struct.set_num_used_workers_isSet(true);
+      struct.supervisor_id = iprot.readString();
+      struct.set_supervisor_id_isSet(true);
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.version = iprot.readString();
+        struct.set_version_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map104 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.total_resources = new HashMap<String,Double>(2*_map104.size);
+          String _key105;
+          double _val106;
+          for (int _i107 = 0; _i107 < _map104.size; ++_i107)
+          {
+            _key105 = iprot.readString();
+            _val106 = iprot.readDouble();
+            struct.total_resources.put(_key105, _val106);
+          }
+        }
+        struct.set_total_resources_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.used_mem = iprot.readDouble();
+        struct.set_used_mem_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.used_cpu = iprot.readDouble();
+        struct.set_used_cpu_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ThriftSerializedObject.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ThriftSerializedObject.java b/storm-client/src/jvm/org/apache/storm/generated/ThriftSerializedObject.java
new file mode 100644
index 0000000..4aee162
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ThriftSerializedObject.java
@@ -0,0 +1,516 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ThriftSerializedObject implements org.apache.thrift.TBase<ThriftSerializedObject, ThriftSerializedObject._Fields>, java.io.Serializable, Cloneable, Comparable<ThriftSerializedObject> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ThriftSerializedObject");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField BITS_FIELD_DESC = new org.apache.thrift.protocol.TField("bits", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ThriftSerializedObjectStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ThriftSerializedObjectTupleSchemeFactory());
+  }
+
+  private String name; // required
+  private ByteBuffer bits; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    BITS((short)2, "bits");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // BITS
+          return BITS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.BITS, new org.apache.thrift.meta_data.FieldMetaData("bits", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ThriftSerializedObject.class, metaDataMap);
+  }
+
+  public ThriftSerializedObject() {
+  }
+
+  public ThriftSerializedObject(
+    String name,
+    ByteBuffer bits)
+  {
+    this();
+    this.name = name;
+    this.bits = org.apache.thrift.TBaseHelper.copyBinary(bits);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ThriftSerializedObject(ThriftSerializedObject other) {
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    if (other.is_set_bits()) {
+      this.bits = org.apache.thrift.TBaseHelper.copyBinary(other.bits);
+    }
+  }
+
+  public ThriftSerializedObject deepCopy() {
+    return new ThriftSerializedObject(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.bits = null;
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public byte[] get_bits() {
+    set_bits(org.apache.thrift.TBaseHelper.rightSize(bits));
+    return bits == null ? null : bits.array();
+  }
+
+  public ByteBuffer buffer_for_bits() {
+    return org.apache.thrift.TBaseHelper.copyBinary(bits);
+  }
+
+  public void set_bits(byte[] bits) {
+    this.bits = bits == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(bits, bits.length));
+  }
+
+  public void set_bits(ByteBuffer bits) {
+    this.bits = org.apache.thrift.TBaseHelper.copyBinary(bits);
+  }
+
+  public void unset_bits() {
+    this.bits = null;
+  }
+
+  /** Returns true if field bits is set (has been assigned a value) and false otherwise */
+  public boolean is_set_bits() {
+    return this.bits != null;
+  }
+
+  public void set_bits_isSet(boolean value) {
+    if (!value) {
+      this.bits = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case BITS:
+      if (value == null) {
+        unset_bits();
+      } else {
+        set_bits((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return get_name();
+
+    case BITS:
+      return get_bits();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return is_set_name();
+    case BITS:
+      return is_set_bits();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ThriftSerializedObject)
+      return this.equals((ThriftSerializedObject)that);
+    return false;
+  }
+
+  public boolean equals(ThriftSerializedObject that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_bits = true && this.is_set_bits();
+    boolean that_present_bits = true && that.is_set_bits();
+    if (this_present_bits || that_present_bits) {
+      if (!(this_present_bits && that_present_bits))
+        return false;
+      if (!this.bits.equals(that.bits))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_bits = true && (is_set_bits());
+    list.add(present_bits);
+    if (present_bits)
+      list.add(bits);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ThriftSerializedObject other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_bits()).compareTo(other.is_set_bits());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_bits()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bits, other.bits);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ThriftSerializedObject(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bits:");
+    if (this.bits == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.bits, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_bits()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'bits' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ThriftSerializedObjectStandardSchemeFactory implements SchemeFactory {
+    public ThriftSerializedObjectStandardScheme getScheme() {
+      return new ThriftSerializedObjectStandardScheme();
+    }
+  }
+
+  private static class ThriftSerializedObjectStandardScheme extends StandardScheme<ThriftSerializedObject> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ThriftSerializedObject struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // BITS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.bits = iprot.readBinary();
+              struct.set_bits_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ThriftSerializedObject struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.bits != null) {
+        oprot.writeFieldBegin(BITS_FIELD_DESC);
+        oprot.writeBinary(struct.bits);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ThriftSerializedObjectTupleSchemeFactory implements SchemeFactory {
+    public ThriftSerializedObjectTupleScheme getScheme() {
+      return new ThriftSerializedObjectTupleScheme();
+    }
+  }
+
+  private static class ThriftSerializedObjectTupleScheme extends TupleScheme<ThriftSerializedObject> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ThriftSerializedObject struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.name);
+      oprot.writeBinary(struct.bits);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ThriftSerializedObject struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.name = iprot.readString();
+      struct.set_name_isSet(true);
+      struct.bits = iprot.readBinary();
+      struct.set_bits_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologyActionOptions.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyActionOptions.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyActionOptions.java
new file mode 100644
index 0000000..1b6b69d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyActionOptions.java
@@ -0,0 +1,387 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class TopologyActionOptions extends org.apache.thrift.TUnion<TopologyActionOptions, TopologyActionOptions._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyActionOptions");
+  private static final org.apache.thrift.protocol.TField KILL_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("kill_options", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField REBALANCE_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("rebalance_options", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    KILL_OPTIONS((short)1, "kill_options"),
+    REBALANCE_OPTIONS((short)2, "rebalance_options");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // KILL_OPTIONS
+          return KILL_OPTIONS;
+        case 2: // REBALANCE_OPTIONS
+          return REBALANCE_OPTIONS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.KILL_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("kill_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KillOptions.class)));
+    tmpMap.put(_Fields.REBALANCE_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("rebalance_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RebalanceOptions.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyActionOptions.class, metaDataMap);
+  }
+
+  public TopologyActionOptions() {
+    super();
+  }
+
+  public TopologyActionOptions(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public TopologyActionOptions(TopologyActionOptions other) {
+    super(other);
+  }
+  public TopologyActionOptions deepCopy() {
+    return new TopologyActionOptions(this);
+  }
+
+  public static TopologyActionOptions kill_options(KillOptions value) {
+    TopologyActionOptions x = new TopologyActionOptions();
+    x.set_kill_options(value);
+    return x;
+  }
+
+  public static TopologyActionOptions rebalance_options(RebalanceOptions value) {
+    TopologyActionOptions x = new TopologyActionOptions();
+    x.set_rebalance_options(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case KILL_OPTIONS:
+        if (value instanceof KillOptions) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type KillOptions for field 'kill_options', but got " + value.getClass().getSimpleName());
+      case REBALANCE_OPTIONS:
+        if (value instanceof RebalanceOptions) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type RebalanceOptions for field 'rebalance_options', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case KILL_OPTIONS:
+          if (field.type == KILL_OPTIONS_FIELD_DESC.type) {
+            KillOptions kill_options;
+            kill_options = new KillOptions();
+            kill_options.read(iprot);
+            return kill_options;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case REBALANCE_OPTIONS:
+          if (field.type == REBALANCE_OPTIONS_FIELD_DESC.type) {
+            RebalanceOptions rebalance_options;
+            rebalance_options = new RebalanceOptions();
+            rebalance_options.read(iprot);
+            return rebalance_options;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case KILL_OPTIONS:
+        KillOptions kill_options = (KillOptions)value_;
+        kill_options.write(oprot);
+        return;
+      case REBALANCE_OPTIONS:
+        RebalanceOptions rebalance_options = (RebalanceOptions)value_;
+        rebalance_options.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case KILL_OPTIONS:
+          KillOptions kill_options;
+          kill_options = new KillOptions();
+          kill_options.read(iprot);
+          return kill_options;
+        case REBALANCE_OPTIONS:
+          RebalanceOptions rebalance_options;
+          rebalance_options = new RebalanceOptions();
+          rebalance_options.read(iprot);
+          return rebalance_options;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case KILL_OPTIONS:
+        KillOptions kill_options = (KillOptions)value_;
+        kill_options.write(oprot);
+        return;
+      case REBALANCE_OPTIONS:
+        RebalanceOptions rebalance_options = (RebalanceOptions)value_;
+        rebalance_options.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case KILL_OPTIONS:
+        return KILL_OPTIONS_FIELD_DESC;
+      case REBALANCE_OPTIONS:
+        return REBALANCE_OPTIONS_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public KillOptions get_kill_options() {
+    if (getSetField() == _Fields.KILL_OPTIONS) {
+      return (KillOptions)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'kill_options' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_kill_options(KillOptions value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.KILL_OPTIONS;
+    value_ = value;
+  }
+
+  public RebalanceOptions get_rebalance_options() {
+    if (getSetField() == _Fields.REBALANCE_OPTIONS) {
+      return (RebalanceOptions)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'rebalance_options' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_rebalance_options(RebalanceOptions value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.REBALANCE_OPTIONS;
+    value_ = value;
+  }
+
+  public boolean is_set_kill_options() {
+    return setField_ == _Fields.KILL_OPTIONS;
+  }
+
+
+  public boolean is_set_rebalance_options() {
+    return setField_ == _Fields.REBALANCE_OPTIONS;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof TopologyActionOptions) {
+      return equals((TopologyActionOptions)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(TopologyActionOptions other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(TopologyActionOptions other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}


[24/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java b/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java
new file mode 100644
index 0000000..6a6d4f2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java
@@ -0,0 +1,466 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._Fields>, java.io.Serializable, Cloneable, Comparable<HBRecords> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBRecords");
+
+  private static final org.apache.thrift.protocol.TField PULSES_FIELD_DESC = new org.apache.thrift.protocol.TField("pulses", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBRecordsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBRecordsTupleSchemeFactory());
+  }
+
+  private List<HBPulse> pulses; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PULSES((short)1, "pulses");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PULSES
+          return PULSES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PULSES, new org.apache.thrift.meta_data.FieldMetaData("pulses", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBPulse.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBRecords.class, metaDataMap);
+  }
+
+  public HBRecords() {
+  }
+
+  public HBRecords(
+    List<HBPulse> pulses)
+  {
+    this();
+    this.pulses = pulses;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBRecords(HBRecords other) {
+    if (other.is_set_pulses()) {
+      List<HBPulse> __this__pulses = new ArrayList<HBPulse>(other.pulses.size());
+      for (HBPulse other_element : other.pulses) {
+        __this__pulses.add(new HBPulse(other_element));
+      }
+      this.pulses = __this__pulses;
+    }
+  }
+
+  public HBRecords deepCopy() {
+    return new HBRecords(this);
+  }
+
+  @Override
+  public void clear() {
+    this.pulses = null;
+  }
+
+  public int get_pulses_size() {
+    return (this.pulses == null) ? 0 : this.pulses.size();
+  }
+
+  public java.util.Iterator<HBPulse> get_pulses_iterator() {
+    return (this.pulses == null) ? null : this.pulses.iterator();
+  }
+
+  public void add_to_pulses(HBPulse elem) {
+    if (this.pulses == null) {
+      this.pulses = new ArrayList<HBPulse>();
+    }
+    this.pulses.add(elem);
+  }
+
+  public List<HBPulse> get_pulses() {
+    return this.pulses;
+  }
+
+  public void set_pulses(List<HBPulse> pulses) {
+    this.pulses = pulses;
+  }
+
+  public void unset_pulses() {
+    this.pulses = null;
+  }
+
+  /** Returns true if field pulses is set (has been assigned a value) and false otherwise */
+  public boolean is_set_pulses() {
+    return this.pulses != null;
+  }
+
+  public void set_pulses_isSet(boolean value) {
+    if (!value) {
+      this.pulses = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PULSES:
+      if (value == null) {
+        unset_pulses();
+      } else {
+        set_pulses((List<HBPulse>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PULSES:
+      return get_pulses();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case PULSES:
+      return is_set_pulses();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBRecords)
+      return this.equals((HBRecords)that);
+    return false;
+  }
+
+  public boolean equals(HBRecords that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_pulses = true && this.is_set_pulses();
+    boolean that_present_pulses = true && that.is_set_pulses();
+    if (this_present_pulses || that_present_pulses) {
+      if (!(this_present_pulses && that_present_pulses))
+        return false;
+      if (!this.pulses.equals(that.pulses))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_pulses = true && (is_set_pulses());
+    list.add(present_pulses);
+    if (present_pulses)
+      list.add(pulses);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBRecords other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_pulses()).compareTo(other.is_set_pulses());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_pulses()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pulses, other.pulses);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBRecords(");
+    boolean first = true;
+
+    sb.append("pulses:");
+    if (this.pulses == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pulses);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBRecordsStandardSchemeFactory implements SchemeFactory {
+    public HBRecordsStandardScheme getScheme() {
+      return new HBRecordsStandardScheme();
+    }
+  }
+
+  private static class HBRecordsStandardScheme extends StandardScheme<HBRecords> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBRecords struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // PULSES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list792 = iprot.readListBegin();
+                struct.pulses = new ArrayList<HBPulse>(_list792.size);
+                HBPulse _elem793;
+                for (int _i794 = 0; _i794 < _list792.size; ++_i794)
+                {
+                  _elem793 = new HBPulse();
+                  _elem793.read(iprot);
+                  struct.pulses.add(_elem793);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_pulses_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBRecords struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.pulses != null) {
+        oprot.writeFieldBegin(PULSES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pulses.size()));
+          for (HBPulse _iter795 : struct.pulses)
+          {
+            _iter795.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBRecordsTupleSchemeFactory implements SchemeFactory {
+    public HBRecordsTupleScheme getScheme() {
+      return new HBRecordsTupleScheme();
+    }
+  }
+
+  private static class HBRecordsTupleScheme extends TupleScheme<HBRecords> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBRecords struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_pulses()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_pulses()) {
+        {
+          oprot.writeI32(struct.pulses.size());
+          for (HBPulse _iter796 : struct.pulses)
+          {
+            _iter796.write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBRecords struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.pulses = new ArrayList<HBPulse>(_list797.size);
+          HBPulse _elem798;
+          for (int _i799 = 0; _i799 < _list797.size; ++_i799)
+          {
+            _elem798 = new HBPulse();
+            _elem798.read(iprot);
+            struct.pulses.add(_elem798);
+          }
+        }
+        struct.set_pulses_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBServerMessageType.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBServerMessageType.java b/storm-client/src/jvm/org/apache/storm/generated/HBServerMessageType.java
new file mode 100644
index 0000000..81e0826
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBServerMessageType.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum HBServerMessageType implements org.apache.thrift.TEnum {
+  CREATE_PATH(0),
+  CREATE_PATH_RESPONSE(1),
+  EXISTS(2),
+  EXISTS_RESPONSE(3),
+  SEND_PULSE(4),
+  SEND_PULSE_RESPONSE(5),
+  GET_ALL_PULSE_FOR_PATH(6),
+  GET_ALL_PULSE_FOR_PATH_RESPONSE(7),
+  GET_ALL_NODES_FOR_PATH(8),
+  GET_ALL_NODES_FOR_PATH_RESPONSE(9),
+  GET_PULSE(10),
+  GET_PULSE_RESPONSE(11),
+  DELETE_PATH(12),
+  DELETE_PATH_RESPONSE(13),
+  DELETE_PULSE_ID(14),
+  DELETE_PULSE_ID_RESPONSE(15),
+  CONTROL_MESSAGE(16),
+  SASL_MESSAGE_TOKEN(17),
+  NOT_AUTHORIZED(18);
+
+  private final int value;
+
+  private HBServerMessageType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static HBServerMessageType findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return CREATE_PATH;
+      case 1:
+        return CREATE_PATH_RESPONSE;
+      case 2:
+        return EXISTS;
+      case 3:
+        return EXISTS_RESPONSE;
+      case 4:
+        return SEND_PULSE;
+      case 5:
+        return SEND_PULSE_RESPONSE;
+      case 6:
+        return GET_ALL_PULSE_FOR_PATH;
+      case 7:
+        return GET_ALL_PULSE_FOR_PATH_RESPONSE;
+      case 8:
+        return GET_ALL_NODES_FOR_PATH;
+      case 9:
+        return GET_ALL_NODES_FOR_PATH_RESPONSE;
+      case 10:
+        return GET_PULSE;
+      case 11:
+        return GET_PULSE_RESPONSE;
+      case 12:
+        return DELETE_PATH;
+      case 13:
+        return DELETE_PATH_RESPONSE;
+      case 14:
+        return DELETE_PULSE_ID;
+      case 15:
+        return DELETE_PULSE_ID_RESPONSE;
+      case 16:
+        return CONTROL_MESSAGE;
+      case 17:
+        return SASL_MESSAGE_TOKEN;
+      case 18:
+        return NOT_AUTHORIZED;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/InvalidTopologyException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/InvalidTopologyException.java b/storm-client/src/jvm/org/apache/storm/generated/InvalidTopologyException.java
new file mode 100644
index 0000000..9591412
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/InvalidTopologyException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class InvalidTopologyException extends TException implements org.apache.thrift.TBase<InvalidTopologyException, InvalidTopologyException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidTopologyException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidTopologyException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new InvalidTopologyExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new InvalidTopologyExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(InvalidTopologyException.class, metaDataMap);
+  }
+
+  public InvalidTopologyException() {
+  }
+
+  public InvalidTopologyException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public InvalidTopologyException(InvalidTopologyException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public InvalidTopologyException deepCopy() {
+    return new InvalidTopologyException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof InvalidTopologyException)
+      return this.equals((InvalidTopologyException)that);
+    return false;
+  }
+
+  public boolean equals(InvalidTopologyException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(InvalidTopologyException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("InvalidTopologyException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class InvalidTopologyExceptionStandardSchemeFactory implements SchemeFactory {
+    public InvalidTopologyExceptionStandardScheme getScheme() {
+      return new InvalidTopologyExceptionStandardScheme();
+    }
+  }
+
+  private static class InvalidTopologyExceptionStandardScheme extends StandardScheme<InvalidTopologyException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, InvalidTopologyException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, InvalidTopologyException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class InvalidTopologyExceptionTupleSchemeFactory implements SchemeFactory {
+    public InvalidTopologyExceptionTupleScheme getScheme() {
+      return new InvalidTopologyExceptionTupleScheme();
+    }
+  }
+
+  private static class InvalidTopologyExceptionTupleScheme extends TupleScheme<InvalidTopologyException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, InvalidTopologyException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, InvalidTopologyException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/JavaObject.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/JavaObject.java b/storm-client/src/jvm/org/apache/storm/generated/JavaObject.java
new file mode 100644
index 0000000..4d7af9d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/JavaObject.java
@@ -0,0 +1,561 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class JavaObject implements org.apache.thrift.TBase<JavaObject, JavaObject._Fields>, java.io.Serializable, Cloneable, Comparable<JavaObject> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("JavaObject");
+
+  private static final org.apache.thrift.protocol.TField FULL_CLASS_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("full_class_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField ARGS_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("args_list", org.apache.thrift.protocol.TType.LIST, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new JavaObjectStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new JavaObjectTupleSchemeFactory());
+  }
+
+  private String full_class_name; // required
+  private List<JavaObjectArg> args_list; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FULL_CLASS_NAME((short)1, "full_class_name"),
+    ARGS_LIST((short)2, "args_list");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FULL_CLASS_NAME
+          return FULL_CLASS_NAME;
+        case 2: // ARGS_LIST
+          return ARGS_LIST;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FULL_CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("full_class_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ARGS_LIST, new org.apache.thrift.meta_data.FieldMetaData("args_list", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, JavaObjectArg.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(JavaObject.class, metaDataMap);
+  }
+
+  public JavaObject() {
+  }
+
+  public JavaObject(
+    String full_class_name,
+    List<JavaObjectArg> args_list)
+  {
+    this();
+    this.full_class_name = full_class_name;
+    this.args_list = args_list;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public JavaObject(JavaObject other) {
+    if (other.is_set_full_class_name()) {
+      this.full_class_name = other.full_class_name;
+    }
+    if (other.is_set_args_list()) {
+      List<JavaObjectArg> __this__args_list = new ArrayList<JavaObjectArg>(other.args_list.size());
+      for (JavaObjectArg other_element : other.args_list) {
+        __this__args_list.add(new JavaObjectArg(other_element));
+      }
+      this.args_list = __this__args_list;
+    }
+  }
+
+  public JavaObject deepCopy() {
+    return new JavaObject(this);
+  }
+
+  @Override
+  public void clear() {
+    this.full_class_name = null;
+    this.args_list = null;
+  }
+
+  public String get_full_class_name() {
+    return this.full_class_name;
+  }
+
+  public void set_full_class_name(String full_class_name) {
+    this.full_class_name = full_class_name;
+  }
+
+  public void unset_full_class_name() {
+    this.full_class_name = null;
+  }
+
+  /** Returns true if field full_class_name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_full_class_name() {
+    return this.full_class_name != null;
+  }
+
+  public void set_full_class_name_isSet(boolean value) {
+    if (!value) {
+      this.full_class_name = null;
+    }
+  }
+
+  public int get_args_list_size() {
+    return (this.args_list == null) ? 0 : this.args_list.size();
+  }
+
+  public java.util.Iterator<JavaObjectArg> get_args_list_iterator() {
+    return (this.args_list == null) ? null : this.args_list.iterator();
+  }
+
+  public void add_to_args_list(JavaObjectArg elem) {
+    if (this.args_list == null) {
+      this.args_list = new ArrayList<JavaObjectArg>();
+    }
+    this.args_list.add(elem);
+  }
+
+  public List<JavaObjectArg> get_args_list() {
+    return this.args_list;
+  }
+
+  public void set_args_list(List<JavaObjectArg> args_list) {
+    this.args_list = args_list;
+  }
+
+  public void unset_args_list() {
+    this.args_list = null;
+  }
+
+  /** Returns true if field args_list is set (has been assigned a value) and false otherwise */
+  public boolean is_set_args_list() {
+    return this.args_list != null;
+  }
+
+  public void set_args_list_isSet(boolean value) {
+    if (!value) {
+      this.args_list = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FULL_CLASS_NAME:
+      if (value == null) {
+        unset_full_class_name();
+      } else {
+        set_full_class_name((String)value);
+      }
+      break;
+
+    case ARGS_LIST:
+      if (value == null) {
+        unset_args_list();
+      } else {
+        set_args_list((List<JavaObjectArg>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FULL_CLASS_NAME:
+      return get_full_class_name();
+
+    case ARGS_LIST:
+      return get_args_list();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case FULL_CLASS_NAME:
+      return is_set_full_class_name();
+    case ARGS_LIST:
+      return is_set_args_list();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof JavaObject)
+      return this.equals((JavaObject)that);
+    return false;
+  }
+
+  public boolean equals(JavaObject that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_full_class_name = true && this.is_set_full_class_name();
+    boolean that_present_full_class_name = true && that.is_set_full_class_name();
+    if (this_present_full_class_name || that_present_full_class_name) {
+      if (!(this_present_full_class_name && that_present_full_class_name))
+        return false;
+      if (!this.full_class_name.equals(that.full_class_name))
+        return false;
+    }
+
+    boolean this_present_args_list = true && this.is_set_args_list();
+    boolean that_present_args_list = true && that.is_set_args_list();
+    if (this_present_args_list || that_present_args_list) {
+      if (!(this_present_args_list && that_present_args_list))
+        return false;
+      if (!this.args_list.equals(that.args_list))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_full_class_name = true && (is_set_full_class_name());
+    list.add(present_full_class_name);
+    if (present_full_class_name)
+      list.add(full_class_name);
+
+    boolean present_args_list = true && (is_set_args_list());
+    list.add(present_args_list);
+    if (present_args_list)
+      list.add(args_list);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(JavaObject other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_full_class_name()).compareTo(other.is_set_full_class_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_full_class_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.full_class_name, other.full_class_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_args_list()).compareTo(other.is_set_args_list());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_args_list()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.args_list, other.args_list);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("JavaObject(");
+    boolean first = true;
+
+    sb.append("full_class_name:");
+    if (this.full_class_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.full_class_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("args_list:");
+    if (this.args_list == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.args_list);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_full_class_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'full_class_name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_args_list()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'args_list' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class JavaObjectStandardSchemeFactory implements SchemeFactory {
+    public JavaObjectStandardScheme getScheme() {
+      return new JavaObjectStandardScheme();
+    }
+  }
+
+  private static class JavaObjectStandardScheme extends StandardScheme<JavaObject> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, JavaObject struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // FULL_CLASS_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.full_class_name = iprot.readString();
+              struct.set_full_class_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // ARGS_LIST
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
+                struct.args_list = new ArrayList<JavaObjectArg>(_list0.size);
+                JavaObjectArg _elem1;
+                for (int _i2 = 0; _i2 < _list0.size; ++_i2)
+                {
+                  _elem1 = new JavaObjectArg();
+                  _elem1.read(iprot);
+                  struct.args_list.add(_elem1);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_args_list_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, JavaObject struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.full_class_name != null) {
+        oprot.writeFieldBegin(FULL_CLASS_NAME_FIELD_DESC);
+        oprot.writeString(struct.full_class_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.args_list != null) {
+        oprot.writeFieldBegin(ARGS_LIST_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.args_list.size()));
+          for (JavaObjectArg _iter3 : struct.args_list)
+          {
+            _iter3.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class JavaObjectTupleSchemeFactory implements SchemeFactory {
+    public JavaObjectTupleScheme getScheme() {
+      return new JavaObjectTupleScheme();
+    }
+  }
+
+  private static class JavaObjectTupleScheme extends TupleScheme<JavaObject> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, JavaObject struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.full_class_name);
+      {
+        oprot.writeI32(struct.args_list.size());
+        for (JavaObjectArg _iter4 : struct.args_list)
+        {
+          _iter4.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, JavaObject struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.full_class_name = iprot.readString();
+      struct.set_full_class_name_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.args_list = new ArrayList<JavaObjectArg>(_list5.size);
+        JavaObjectArg _elem6;
+        for (int _i7 = 0; _i7 < _list5.size; ++_i7)
+        {
+          _elem6 = new JavaObjectArg();
+          _elem6.read(iprot);
+          struct.args_list.add(_elem6);
+        }
+      }
+      struct.set_args_list_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/JavaObjectArg.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/JavaObjectArg.java b/storm-client/src/jvm/org/apache/storm/generated/JavaObjectArg.java
new file mode 100644
index 0000000..bcb2596
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/JavaObjectArg.java
@@ -0,0 +1,631 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class JavaObjectArg extends org.apache.thrift.TUnion<JavaObjectArg, JavaObjectArg._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("JavaObjectArg");
+  private static final org.apache.thrift.protocol.TField INT_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("int_arg", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField LONG_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("long_arg", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField STRING_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("string_arg", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField BOOL_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("bool_arg", org.apache.thrift.protocol.TType.BOOL, (short)4);
+  private static final org.apache.thrift.protocol.TField BINARY_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("binary_arg", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField DOUBLE_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("double_arg", org.apache.thrift.protocol.TType.DOUBLE, (short)6);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    INT_ARG((short)1, "int_arg"),
+    LONG_ARG((short)2, "long_arg"),
+    STRING_ARG((short)3, "string_arg"),
+    BOOL_ARG((short)4, "bool_arg"),
+    BINARY_ARG((short)5, "binary_arg"),
+    DOUBLE_ARG((short)6, "double_arg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // INT_ARG
+          return INT_ARG;
+        case 2: // LONG_ARG
+          return LONG_ARG;
+        case 3: // STRING_ARG
+          return STRING_ARG;
+        case 4: // BOOL_ARG
+          return BOOL_ARG;
+        case 5: // BINARY_ARG
+          return BINARY_ARG;
+        case 6: // DOUBLE_ARG
+          return DOUBLE_ARG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.INT_ARG, new org.apache.thrift.meta_data.FieldMetaData("int_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.LONG_ARG, new org.apache.thrift.meta_data.FieldMetaData("long_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.STRING_ARG, new org.apache.thrift.meta_data.FieldMetaData("string_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.BOOL_ARG, new org.apache.thrift.meta_data.FieldMetaData("bool_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.BINARY_ARG, new org.apache.thrift.meta_data.FieldMetaData("binary_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.DOUBLE_ARG, new org.apache.thrift.meta_data.FieldMetaData("double_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(JavaObjectArg.class, metaDataMap);
+  }
+
+  public JavaObjectArg() {
+    super();
+  }
+
+  public JavaObjectArg(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public JavaObjectArg(JavaObjectArg other) {
+    super(other);
+  }
+  public JavaObjectArg deepCopy() {
+    return new JavaObjectArg(this);
+  }
+
+  public static JavaObjectArg int_arg(int value) {
+    JavaObjectArg x = new JavaObjectArg();
+    x.set_int_arg(value);
+    return x;
+  }
+
+  public static JavaObjectArg long_arg(long value) {
+    JavaObjectArg x = new JavaObjectArg();
+    x.set_long_arg(value);
+    return x;
+  }
+
+  public static JavaObjectArg string_arg(String value) {
+    JavaObjectArg x = new JavaObjectArg();
+    x.set_string_arg(value);
+    return x;
+  }
+
+  public static JavaObjectArg bool_arg(boolean value) {
+    JavaObjectArg x = new JavaObjectArg();
+    x.set_bool_arg(value);
+    return x;
+  }
+
+  public static JavaObjectArg binary_arg(ByteBuffer value) {
+    JavaObjectArg x = new JavaObjectArg();
+    x.set_binary_arg(value);
+    return x;
+  }
+
+  public static JavaObjectArg binary_arg(byte[] value) {
+    JavaObjectArg x = new JavaObjectArg();
+    x.set_binary_arg(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+    return x;
+  }
+
+  public static JavaObjectArg double_arg(double value) {
+    JavaObjectArg x = new JavaObjectArg();
+    x.set_double_arg(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case INT_ARG:
+        if (value instanceof Integer) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type Integer for field 'int_arg', but got " + value.getClass().getSimpleName());
+      case LONG_ARG:
+        if (value instanceof Long) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type Long for field 'long_arg', but got " + value.getClass().getSimpleName());
+      case STRING_ARG:
+        if (value instanceof String) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type String for field 'string_arg', but got " + value.getClass().getSimpleName());
+      case BOOL_ARG:
+        if (value instanceof Boolean) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type Boolean for field 'bool_arg', but got " + value.getClass().getSimpleName());
+      case BINARY_ARG:
+        if (value instanceof ByteBuffer) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type ByteBuffer for field 'binary_arg', but got " + value.getClass().getSimpleName());
+      case DOUBLE_ARG:
+        if (value instanceof Double) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type Double for field 'double_arg', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case INT_ARG:
+          if (field.type == INT_ARG_FIELD_DESC.type) {
+            Integer int_arg;
+            int_arg = iprot.readI32();
+            return int_arg;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case LONG_ARG:
+          if (field.type == LONG_ARG_FIELD_DESC.type) {
+            Long long_arg;
+            long_arg = iprot.readI64();
+            return long_arg;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case STRING_ARG:
+          if (field.type == STRING_ARG_FIELD_DESC.type) {
+            String string_arg;
+            string_arg = iprot.readString();
+            return string_arg;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case BOOL_ARG:
+          if (field.type == BOOL_ARG_FIELD_DESC.type) {
+            Boolean bool_arg;
+            bool_arg = iprot.readBool();
+            return bool_arg;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case BINARY_ARG:
+          if (field.type == BINARY_ARG_FIELD_DESC.type) {
+            ByteBuffer binary_arg;
+            binary_arg = iprot.readBinary();
+            return binary_arg;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case DOUBLE_ARG:
+          if (field.type == DOUBLE_ARG_FIELD_DESC.type) {
+            Double double_arg;
+            double_arg = iprot.readDouble();
+            return double_arg;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case INT_ARG:
+        Integer int_arg = (Integer)value_;
+        oprot.writeI32(int_arg);
+        return;
+      case LONG_ARG:
+        Long long_arg = (Long)value_;
+        oprot.writeI64(long_arg);
+        return;
+      case STRING_ARG:
+        String string_arg = (String)value_;
+        oprot.writeString(string_arg);
+        return;
+      case BOOL_ARG:
+        Boolean bool_arg = (Boolean)value_;
+        oprot.writeBool(bool_arg);
+        return;
+      case BINARY_ARG:
+        ByteBuffer binary_arg = (ByteBuffer)value_;
+        oprot.writeBinary(binary_arg);
+        return;
+      case DOUBLE_ARG:
+        Double double_arg = (Double)value_;
+        oprot.writeDouble(double_arg);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case INT_ARG:
+          Integer int_arg;
+          int_arg = iprot.readI32();
+          return int_arg;
+        case LONG_ARG:
+          Long long_arg;
+          long_arg = iprot.readI64();
+          return long_arg;
+        case STRING_ARG:
+          String string_arg;
+          string_arg = iprot.readString();
+          return string_arg;
+        case BOOL_ARG:
+          Boolean bool_arg;
+          bool_arg = iprot.readBool();
+          return bool_arg;
+        case BINARY_ARG:
+          ByteBuffer binary_arg;
+          binary_arg = iprot.readBinary();
+          return binary_arg;
+        case DOUBLE_ARG:
+          Double double_arg;
+          double_arg = iprot.readDouble();
+          return double_arg;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case INT_ARG:
+        Integer int_arg = (Integer)value_;
+        oprot.writeI32(int_arg);
+        return;
+      case LONG_ARG:
+        Long long_arg = (Long)value_;
+        oprot.writeI64(long_arg);
+        return;
+      case STRING_ARG:
+        String string_arg = (String)value_;
+        oprot.writeString(string_arg);
+        return;
+      case BOOL_ARG:
+        Boolean bool_arg = (Boolean)value_;
+        oprot.writeBool(bool_arg);
+        return;
+      case BINARY_ARG:
+        ByteBuffer binary_arg = (ByteBuffer)value_;
+        oprot.writeBinary(binary_arg);
+        return;
+      case DOUBLE_ARG:
+        Double double_arg = (Double)value_;
+        oprot.writeDouble(double_arg);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case INT_ARG:
+        return INT_ARG_FIELD_DESC;
+      case LONG_ARG:
+        return LONG_ARG_FIELD_DESC;
+      case STRING_ARG:
+        return STRING_ARG_FIELD_DESC;
+      case BOOL_ARG:
+        return BOOL_ARG_FIELD_DESC;
+      case BINARY_ARG:
+        return BINARY_ARG_FIELD_DESC;
+      case DOUBLE_ARG:
+        return DOUBLE_ARG_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public int get_int_arg() {
+    if (getSetField() == _Fields.INT_ARG) {
+      return (Integer)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'int_arg' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_int_arg(int value) {
+    setField_ = _Fields.INT_ARG;
+    value_ = value;
+  }
+
+  public long get_long_arg() {
+    if (getSetField() == _Fields.LONG_ARG) {
+      return (Long)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'long_arg' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_long_arg(long value) {
+    setField_ = _Fields.LONG_ARG;
+    value_ = value;
+  }
+
+  public String get_string_arg() {
+    if (getSetField() == _Fields.STRING_ARG) {
+      return (String)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'string_arg' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_string_arg(String value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.STRING_ARG;
+    value_ = value;
+  }
+
+  public boolean get_bool_arg() {
+    if (getSetField() == _Fields.BOOL_ARG) {
+      return (Boolean)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'bool_arg' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_bool_arg(boolean value) {
+    setField_ = _Fields.BOOL_ARG;
+    value_ = value;
+  }
+
+  public byte[] get_binary_arg() {
+    set_binary_arg(org.apache.thrift.TBaseHelper.rightSize(buffer_for_binary_arg()));
+    ByteBuffer b = buffer_for_binary_arg();
+    return b == null ? null : b.array();
+  }
+
+  public ByteBuffer buffer_for_binary_arg() {
+    if (getSetField() == _Fields.BINARY_ARG) {
+      return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue());
+    } else {
+      throw new RuntimeException("Cannot get field 'binary_arg' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_binary_arg(byte[] value) {
+    set_binary_arg(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+  }
+
+  public void set_binary_arg(ByteBuffer value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.BINARY_ARG;
+    value_ = value;
+  }
+
+  public double get_double_arg() {
+    if (getSetField() == _Fields.DOUBLE_ARG) {
+      return (Double)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'double_arg' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_double_arg(double value) {
+    setField_ = _Fields.DOUBLE_ARG;
+    value_ = value;
+  }
+
+  public boolean is_set_int_arg() {
+    return setField_ == _Fields.INT_ARG;
+  }
+
+
+  public boolean is_set_long_arg() {
+    return setField_ == _Fields.LONG_ARG;
+  }
+
+
+  public boolean is_set_string_arg() {
+    return setField_ == _Fields.STRING_ARG;
+  }
+
+
+  public boolean is_set_bool_arg() {
+    return setField_ == _Fields.BOOL_ARG;
+  }
+
+
+  public boolean is_set_binary_arg() {
+    return setField_ == _Fields.BINARY_ARG;
+  }
+
+
+  public boolean is_set_double_arg() {
+    return setField_ == _Fields.DOUBLE_ARG;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof JavaObjectArg) {
+      return equals((JavaObjectArg)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(JavaObjectArg other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(JavaObjectArg other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/KeyAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/KeyAlreadyExistsException.java b/storm-client/src/jvm/org/apache/storm/generated/KeyAlreadyExistsException.java
new file mode 100644
index 0000000..32b0a74
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/KeyAlreadyExistsException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class KeyAlreadyExistsException extends TException implements org.apache.thrift.TBase<KeyAlreadyExistsException, KeyAlreadyExistsException._Fields>, java.io.Serializable, Cloneable, Comparable<KeyAlreadyExistsException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KeyAlreadyExistsException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new KeyAlreadyExistsExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new KeyAlreadyExistsExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KeyAlreadyExistsException.class, metaDataMap);
+  }
+
+  public KeyAlreadyExistsException() {
+  }
+
+  public KeyAlreadyExistsException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public KeyAlreadyExistsException(KeyAlreadyExistsException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public KeyAlreadyExistsException deepCopy() {
+    return new KeyAlreadyExistsException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof KeyAlreadyExistsException)
+      return this.equals((KeyAlreadyExistsException)that);
+    return false;
+  }
+
+  public boolean equals(KeyAlreadyExistsException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(KeyAlreadyExistsException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("KeyAlreadyExistsException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class KeyAlreadyExistsExceptionStandardSchemeFactory implements SchemeFactory {
+    public KeyAlreadyExistsExceptionStandardScheme getScheme() {
+      return new KeyAlreadyExistsExceptionStandardScheme();
+    }
+  }
+
+  private static class KeyAlreadyExistsExceptionStandardScheme extends StandardScheme<KeyAlreadyExistsException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class KeyAlreadyExistsExceptionTupleSchemeFactory implements SchemeFactory {
+    public KeyAlreadyExistsExceptionTupleScheme getScheme() {
+      return new KeyAlreadyExistsExceptionTupleScheme();
+    }
+  }
+
+  private static class KeyAlreadyExistsExceptionTupleScheme extends TupleScheme<KeyAlreadyExistsException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+


[25/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBMessage.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBMessage.java b/storm-client/src/jvm/org/apache/storm/generated/HBMessage.java
new file mode 100644
index 0000000..29d3b21
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBMessage.java
@@ -0,0 +1,636 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBMessage implements org.apache.thrift.TBase<HBMessage, HBMessage._Fields>, java.io.Serializable, Cloneable, Comparable<HBMessage> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBMessage");
+
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("data", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField MESSAGE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("message_id", org.apache.thrift.protocol.TType.I32, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBMessageStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBMessageTupleSchemeFactory());
+  }
+
+  private HBServerMessageType type; // required
+  private HBMessageData data; // required
+  private int message_id; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see HBServerMessageType
+     */
+    TYPE((short)1, "type"),
+    DATA((short)2, "data"),
+    MESSAGE_ID((short)3, "message_id");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TYPE
+          return TYPE;
+        case 2: // DATA
+          return DATA;
+        case 3: // MESSAGE_ID
+          return MESSAGE_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __MESSAGE_ID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.MESSAGE_ID};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, HBServerMessageType.class)));
+    tmpMap.put(_Fields.DATA, new org.apache.thrift.meta_data.FieldMetaData("data", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBMessageData.class)));
+    tmpMap.put(_Fields.MESSAGE_ID, new org.apache.thrift.meta_data.FieldMetaData("message_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBMessage.class, metaDataMap);
+  }
+
+  public HBMessage() {
+    this.message_id = -1;
+
+  }
+
+  public HBMessage(
+    HBServerMessageType type,
+    HBMessageData data)
+  {
+    this();
+    this.type = type;
+    this.data = data;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBMessage(HBMessage other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_type()) {
+      this.type = other.type;
+    }
+    if (other.is_set_data()) {
+      this.data = new HBMessageData(other.data);
+    }
+    this.message_id = other.message_id;
+  }
+
+  public HBMessage deepCopy() {
+    return new HBMessage(this);
+  }
+
+  @Override
+  public void clear() {
+    this.type = null;
+    this.data = null;
+    this.message_id = -1;
+
+  }
+
+  /**
+   * 
+   * @see HBServerMessageType
+   */
+  public HBServerMessageType get_type() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see HBServerMessageType
+   */
+  public void set_type(HBServerMessageType type) {
+    this.type = type;
+  }
+
+  public void unset_type() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_type() {
+    return this.type != null;
+  }
+
+  public void set_type_isSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public HBMessageData get_data() {
+    return this.data;
+  }
+
+  public void set_data(HBMessageData data) {
+    this.data = data;
+  }
+
+  public void unset_data() {
+    this.data = null;
+  }
+
+  /** Returns true if field data is set (has been assigned a value) and false otherwise */
+  public boolean is_set_data() {
+    return this.data != null;
+  }
+
+  public void set_data_isSet(boolean value) {
+    if (!value) {
+      this.data = null;
+    }
+  }
+
+  public int get_message_id() {
+    return this.message_id;
+  }
+
+  public void set_message_id(int message_id) {
+    this.message_id = message_id;
+    set_message_id_isSet(true);
+  }
+
+  public void unset_message_id() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MESSAGE_ID_ISSET_ID);
+  }
+
+  /** Returns true if field message_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_message_id() {
+    return EncodingUtils.testBit(__isset_bitfield, __MESSAGE_ID_ISSET_ID);
+  }
+
+  public void set_message_id_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MESSAGE_ID_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TYPE:
+      if (value == null) {
+        unset_type();
+      } else {
+        set_type((HBServerMessageType)value);
+      }
+      break;
+
+    case DATA:
+      if (value == null) {
+        unset_data();
+      } else {
+        set_data((HBMessageData)value);
+      }
+      break;
+
+    case MESSAGE_ID:
+      if (value == null) {
+        unset_message_id();
+      } else {
+        set_message_id((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TYPE:
+      return get_type();
+
+    case DATA:
+      return get_data();
+
+    case MESSAGE_ID:
+      return get_message_id();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TYPE:
+      return is_set_type();
+    case DATA:
+      return is_set_data();
+    case MESSAGE_ID:
+      return is_set_message_id();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBMessage)
+      return this.equals((HBMessage)that);
+    return false;
+  }
+
+  public boolean equals(HBMessage that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_type = true && this.is_set_type();
+    boolean that_present_type = true && that.is_set_type();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_data = true && this.is_set_data();
+    boolean that_present_data = true && that.is_set_data();
+    if (this_present_data || that_present_data) {
+      if (!(this_present_data && that_present_data))
+        return false;
+      if (!this.data.equals(that.data))
+        return false;
+    }
+
+    boolean this_present_message_id = true && this.is_set_message_id();
+    boolean that_present_message_id = true && that.is_set_message_id();
+    if (this_present_message_id || that_present_message_id) {
+      if (!(this_present_message_id && that_present_message_id))
+        return false;
+      if (this.message_id != that.message_id)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_type = true && (is_set_type());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
+    boolean present_data = true && (is_set_data());
+    list.add(present_data);
+    if (present_data)
+      list.add(data);
+
+    boolean present_message_id = true && (is_set_message_id());
+    list.add(present_message_id);
+    if (present_message_id)
+      list.add(message_id);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBMessage other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_data()).compareTo(other.is_set_data());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_data()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data, other.data);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_message_id()).compareTo(other.is_set_message_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_message_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message_id, other.message_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBMessage(");
+    boolean first = true;
+
+    sb.append("type:");
+    if (this.type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.type);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("data:");
+    if (this.data == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.data);
+    }
+    first = false;
+    if (is_set_message_id()) {
+      if (!first) sb.append(", ");
+      sb.append("message_id:");
+      sb.append(this.message_id);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBMessageStandardSchemeFactory implements SchemeFactory {
+    public HBMessageStandardScheme getScheme() {
+      return new HBMessageStandardScheme();
+    }
+  }
+
+  private static class HBMessageStandardScheme extends StandardScheme<HBMessage> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBMessage struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = org.apache.storm.generated.HBServerMessageType.findByValue(iprot.readI32());
+              struct.set_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DATA
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.data = new HBMessageData();
+              struct.data.read(iprot);
+              struct.set_data_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // MESSAGE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.message_id = iprot.readI32();
+              struct.set_message_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBMessage struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.type != null) {
+        oprot.writeFieldBegin(TYPE_FIELD_DESC);
+        oprot.writeI32(struct.type.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.data != null) {
+        oprot.writeFieldBegin(DATA_FIELD_DESC);
+        struct.data.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_message_id()) {
+        oprot.writeFieldBegin(MESSAGE_ID_FIELD_DESC);
+        oprot.writeI32(struct.message_id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBMessageTupleSchemeFactory implements SchemeFactory {
+    public HBMessageTupleScheme getScheme() {
+      return new HBMessageTupleScheme();
+    }
+  }
+
+  private static class HBMessageTupleScheme extends TupleScheme<HBMessage> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBMessage struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_type()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_data()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_message_id()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_type()) {
+        oprot.writeI32(struct.type.getValue());
+      }
+      if (struct.is_set_data()) {
+        struct.data.write(oprot);
+      }
+      if (struct.is_set_message_id()) {
+        oprot.writeI32(struct.message_id);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBMessage struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.type = org.apache.storm.generated.HBServerMessageType.findByValue(iprot.readI32());
+        struct.set_type_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.data = new HBMessageData();
+        struct.data.read(iprot);
+        struct.set_data_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.message_id = iprot.readI32();
+        struct.set_message_id_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBMessageData.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBMessageData.java b/storm-client/src/jvm/org/apache/storm/generated/HBMessageData.java
new file mode 100644
index 0000000..b100da1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBMessageData.java
@@ -0,0 +1,640 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class HBMessageData extends org.apache.thrift.TUnion<HBMessageData, HBMessageData._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBMessageData");
+  private static final org.apache.thrift.protocol.TField PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("path", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PULSE_FIELD_DESC = new org.apache.thrift.protocol.TField("pulse", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField BOOLVAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolval", org.apache.thrift.protocol.TType.BOOL, (short)3);
+  private static final org.apache.thrift.protocol.TField RECORDS_FIELD_DESC = new org.apache.thrift.protocol.TField("records", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+  private static final org.apache.thrift.protocol.TField NODES_FIELD_DESC = new org.apache.thrift.protocol.TField("nodes", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField MESSAGE_BLOB_FIELD_DESC = new org.apache.thrift.protocol.TField("message_blob", org.apache.thrift.protocol.TType.STRING, (short)7);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PATH((short)1, "path"),
+    PULSE((short)2, "pulse"),
+    BOOLVAL((short)3, "boolval"),
+    RECORDS((short)4, "records"),
+    NODES((short)5, "nodes"),
+    MESSAGE_BLOB((short)7, "message_blob");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PATH
+          return PATH;
+        case 2: // PULSE
+          return PULSE;
+        case 3: // BOOLVAL
+          return BOOLVAL;
+        case 4: // RECORDS
+          return RECORDS;
+        case 5: // NODES
+          return NODES;
+        case 7: // MESSAGE_BLOB
+          return MESSAGE_BLOB;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PATH, new org.apache.thrift.meta_data.FieldMetaData("path", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PULSE, new org.apache.thrift.meta_data.FieldMetaData("pulse", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBPulse.class)));
+    tmpMap.put(_Fields.BOOLVAL, new org.apache.thrift.meta_data.FieldMetaData("boolval", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.RECORDS, new org.apache.thrift.meta_data.FieldMetaData("records", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBRecords.class)));
+    tmpMap.put(_Fields.NODES, new org.apache.thrift.meta_data.FieldMetaData("nodes", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBNodes.class)));
+    tmpMap.put(_Fields.MESSAGE_BLOB, new org.apache.thrift.meta_data.FieldMetaData("message_blob", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBMessageData.class, metaDataMap);
+  }
+
+  public HBMessageData() {
+    super();
+  }
+
+  public HBMessageData(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public HBMessageData(HBMessageData other) {
+    super(other);
+  }
+  public HBMessageData deepCopy() {
+    return new HBMessageData(this);
+  }
+
+  public static HBMessageData path(String value) {
+    HBMessageData x = new HBMessageData();
+    x.set_path(value);
+    return x;
+  }
+
+  public static HBMessageData pulse(HBPulse value) {
+    HBMessageData x = new HBMessageData();
+    x.set_pulse(value);
+    return x;
+  }
+
+  public static HBMessageData boolval(boolean value) {
+    HBMessageData x = new HBMessageData();
+    x.set_boolval(value);
+    return x;
+  }
+
+  public static HBMessageData records(HBRecords value) {
+    HBMessageData x = new HBMessageData();
+    x.set_records(value);
+    return x;
+  }
+
+  public static HBMessageData nodes(HBNodes value) {
+    HBMessageData x = new HBMessageData();
+    x.set_nodes(value);
+    return x;
+  }
+
+  public static HBMessageData message_blob(ByteBuffer value) {
+    HBMessageData x = new HBMessageData();
+    x.set_message_blob(value);
+    return x;
+  }
+
+  public static HBMessageData message_blob(byte[] value) {
+    HBMessageData x = new HBMessageData();
+    x.set_message_blob(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case PATH:
+        if (value instanceof String) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type String for field 'path', but got " + value.getClass().getSimpleName());
+      case PULSE:
+        if (value instanceof HBPulse) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type HBPulse for field 'pulse', but got " + value.getClass().getSimpleName());
+      case BOOLVAL:
+        if (value instanceof Boolean) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type Boolean for field 'boolval', but got " + value.getClass().getSimpleName());
+      case RECORDS:
+        if (value instanceof HBRecords) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type HBRecords for field 'records', but got " + value.getClass().getSimpleName());
+      case NODES:
+        if (value instanceof HBNodes) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type HBNodes for field 'nodes', but got " + value.getClass().getSimpleName());
+      case MESSAGE_BLOB:
+        if (value instanceof ByteBuffer) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type ByteBuffer for field 'message_blob', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case PATH:
+          if (field.type == PATH_FIELD_DESC.type) {
+            String path;
+            path = iprot.readString();
+            return path;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case PULSE:
+          if (field.type == PULSE_FIELD_DESC.type) {
+            HBPulse pulse;
+            pulse = new HBPulse();
+            pulse.read(iprot);
+            return pulse;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case BOOLVAL:
+          if (field.type == BOOLVAL_FIELD_DESC.type) {
+            Boolean boolval;
+            boolval = iprot.readBool();
+            return boolval;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case RECORDS:
+          if (field.type == RECORDS_FIELD_DESC.type) {
+            HBRecords records;
+            records = new HBRecords();
+            records.read(iprot);
+            return records;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case NODES:
+          if (field.type == NODES_FIELD_DESC.type) {
+            HBNodes nodes;
+            nodes = new HBNodes();
+            nodes.read(iprot);
+            return nodes;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case MESSAGE_BLOB:
+          if (field.type == MESSAGE_BLOB_FIELD_DESC.type) {
+            ByteBuffer message_blob;
+            message_blob = iprot.readBinary();
+            return message_blob;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case PATH:
+        String path = (String)value_;
+        oprot.writeString(path);
+        return;
+      case PULSE:
+        HBPulse pulse = (HBPulse)value_;
+        pulse.write(oprot);
+        return;
+      case BOOLVAL:
+        Boolean boolval = (Boolean)value_;
+        oprot.writeBool(boolval);
+        return;
+      case RECORDS:
+        HBRecords records = (HBRecords)value_;
+        records.write(oprot);
+        return;
+      case NODES:
+        HBNodes nodes = (HBNodes)value_;
+        nodes.write(oprot);
+        return;
+      case MESSAGE_BLOB:
+        ByteBuffer message_blob = (ByteBuffer)value_;
+        oprot.writeBinary(message_blob);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case PATH:
+          String path;
+          path = iprot.readString();
+          return path;
+        case PULSE:
+          HBPulse pulse;
+          pulse = new HBPulse();
+          pulse.read(iprot);
+          return pulse;
+        case BOOLVAL:
+          Boolean boolval;
+          boolval = iprot.readBool();
+          return boolval;
+        case RECORDS:
+          HBRecords records;
+          records = new HBRecords();
+          records.read(iprot);
+          return records;
+        case NODES:
+          HBNodes nodes;
+          nodes = new HBNodes();
+          nodes.read(iprot);
+          return nodes;
+        case MESSAGE_BLOB:
+          ByteBuffer message_blob;
+          message_blob = iprot.readBinary();
+          return message_blob;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case PATH:
+        String path = (String)value_;
+        oprot.writeString(path);
+        return;
+      case PULSE:
+        HBPulse pulse = (HBPulse)value_;
+        pulse.write(oprot);
+        return;
+      case BOOLVAL:
+        Boolean boolval = (Boolean)value_;
+        oprot.writeBool(boolval);
+        return;
+      case RECORDS:
+        HBRecords records = (HBRecords)value_;
+        records.write(oprot);
+        return;
+      case NODES:
+        HBNodes nodes = (HBNodes)value_;
+        nodes.write(oprot);
+        return;
+      case MESSAGE_BLOB:
+        ByteBuffer message_blob = (ByteBuffer)value_;
+        oprot.writeBinary(message_blob);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case PATH:
+        return PATH_FIELD_DESC;
+      case PULSE:
+        return PULSE_FIELD_DESC;
+      case BOOLVAL:
+        return BOOLVAL_FIELD_DESC;
+      case RECORDS:
+        return RECORDS_FIELD_DESC;
+      case NODES:
+        return NODES_FIELD_DESC;
+      case MESSAGE_BLOB:
+        return MESSAGE_BLOB_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public String get_path() {
+    if (getSetField() == _Fields.PATH) {
+      return (String)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'path' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_path(String value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.PATH;
+    value_ = value;
+  }
+
+  public HBPulse get_pulse() {
+    if (getSetField() == _Fields.PULSE) {
+      return (HBPulse)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'pulse' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_pulse(HBPulse value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.PULSE;
+    value_ = value;
+  }
+
+  public boolean get_boolval() {
+    if (getSetField() == _Fields.BOOLVAL) {
+      return (Boolean)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'boolval' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_boolval(boolean value) {
+    setField_ = _Fields.BOOLVAL;
+    value_ = value;
+  }
+
+  public HBRecords get_records() {
+    if (getSetField() == _Fields.RECORDS) {
+      return (HBRecords)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'records' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_records(HBRecords value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.RECORDS;
+    value_ = value;
+  }
+
+  public HBNodes get_nodes() {
+    if (getSetField() == _Fields.NODES) {
+      return (HBNodes)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'nodes' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_nodes(HBNodes value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.NODES;
+    value_ = value;
+  }
+
+  public byte[] get_message_blob() {
+    set_message_blob(org.apache.thrift.TBaseHelper.rightSize(buffer_for_message_blob()));
+    ByteBuffer b = buffer_for_message_blob();
+    return b == null ? null : b.array();
+  }
+
+  public ByteBuffer buffer_for_message_blob() {
+    if (getSetField() == _Fields.MESSAGE_BLOB) {
+      return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue());
+    } else {
+      throw new RuntimeException("Cannot get field 'message_blob' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_message_blob(byte[] value) {
+    set_message_blob(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+  }
+
+  public void set_message_blob(ByteBuffer value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.MESSAGE_BLOB;
+    value_ = value;
+  }
+
+  public boolean is_set_path() {
+    return setField_ == _Fields.PATH;
+  }
+
+
+  public boolean is_set_pulse() {
+    return setField_ == _Fields.PULSE;
+  }
+
+
+  public boolean is_set_boolval() {
+    return setField_ == _Fields.BOOLVAL;
+  }
+
+
+  public boolean is_set_records() {
+    return setField_ == _Fields.RECORDS;
+  }
+
+
+  public boolean is_set_nodes() {
+    return setField_ == _Fields.NODES;
+  }
+
+
+  public boolean is_set_message_blob() {
+    return setField_ == _Fields.MESSAGE_BLOB;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof HBMessageData) {
+      return equals((HBMessageData)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(HBMessageData other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(HBMessageData other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java b/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java
new file mode 100644
index 0000000..c5c7407
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java
@@ -0,0 +1,461 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields>, java.io.Serializable, Cloneable, Comparable<HBNodes> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBNodes");
+
+  private static final org.apache.thrift.protocol.TField PULSE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("pulseIds", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBNodesStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBNodesTupleSchemeFactory());
+  }
+
+  private List<String> pulseIds; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PULSE_IDS((short)1, "pulseIds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PULSE_IDS
+          return PULSE_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PULSE_IDS, new org.apache.thrift.meta_data.FieldMetaData("pulseIds", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBNodes.class, metaDataMap);
+  }
+
+  public HBNodes() {
+  }
+
+  public HBNodes(
+    List<String> pulseIds)
+  {
+    this();
+    this.pulseIds = pulseIds;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBNodes(HBNodes other) {
+    if (other.is_set_pulseIds()) {
+      List<String> __this__pulseIds = new ArrayList<String>(other.pulseIds);
+      this.pulseIds = __this__pulseIds;
+    }
+  }
+
+  public HBNodes deepCopy() {
+    return new HBNodes(this);
+  }
+
+  @Override
+  public void clear() {
+    this.pulseIds = null;
+  }
+
+  public int get_pulseIds_size() {
+    return (this.pulseIds == null) ? 0 : this.pulseIds.size();
+  }
+
+  public java.util.Iterator<String> get_pulseIds_iterator() {
+    return (this.pulseIds == null) ? null : this.pulseIds.iterator();
+  }
+
+  public void add_to_pulseIds(String elem) {
+    if (this.pulseIds == null) {
+      this.pulseIds = new ArrayList<String>();
+    }
+    this.pulseIds.add(elem);
+  }
+
+  public List<String> get_pulseIds() {
+    return this.pulseIds;
+  }
+
+  public void set_pulseIds(List<String> pulseIds) {
+    this.pulseIds = pulseIds;
+  }
+
+  public void unset_pulseIds() {
+    this.pulseIds = null;
+  }
+
+  /** Returns true if field pulseIds is set (has been assigned a value) and false otherwise */
+  public boolean is_set_pulseIds() {
+    return this.pulseIds != null;
+  }
+
+  public void set_pulseIds_isSet(boolean value) {
+    if (!value) {
+      this.pulseIds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PULSE_IDS:
+      if (value == null) {
+        unset_pulseIds();
+      } else {
+        set_pulseIds((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PULSE_IDS:
+      return get_pulseIds();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case PULSE_IDS:
+      return is_set_pulseIds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBNodes)
+      return this.equals((HBNodes)that);
+    return false;
+  }
+
+  public boolean equals(HBNodes that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_pulseIds = true && this.is_set_pulseIds();
+    boolean that_present_pulseIds = true && that.is_set_pulseIds();
+    if (this_present_pulseIds || that_present_pulseIds) {
+      if (!(this_present_pulseIds && that_present_pulseIds))
+        return false;
+      if (!this.pulseIds.equals(that.pulseIds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_pulseIds = true && (is_set_pulseIds());
+    list.add(present_pulseIds);
+    if (present_pulseIds)
+      list.add(pulseIds);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBNodes other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_pulseIds()).compareTo(other.is_set_pulseIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_pulseIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pulseIds, other.pulseIds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBNodes(");
+    boolean first = true;
+
+    sb.append("pulseIds:");
+    if (this.pulseIds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pulseIds);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBNodesStandardSchemeFactory implements SchemeFactory {
+    public HBNodesStandardScheme getScheme() {
+      return new HBNodesStandardScheme();
+    }
+  }
+
+  private static class HBNodesStandardScheme extends StandardScheme<HBNodes> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBNodes struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // PULSE_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
+                struct.pulseIds = new ArrayList<String>(_list800.size);
+                String _elem801;
+                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
+                {
+                  _elem801 = iprot.readString();
+                  struct.pulseIds.add(_elem801);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_pulseIds_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBNodes struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.pulseIds != null) {
+        oprot.writeFieldBegin(PULSE_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.pulseIds.size()));
+          for (String _iter803 : struct.pulseIds)
+          {
+            oprot.writeString(_iter803);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBNodesTupleSchemeFactory implements SchemeFactory {
+    public HBNodesTupleScheme getScheme() {
+      return new HBNodesTupleScheme();
+    }
+  }
+
+  private static class HBNodesTupleScheme extends TupleScheme<HBNodes> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBNodes struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_pulseIds()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_pulseIds()) {
+        {
+          oprot.writeI32(struct.pulseIds.size());
+          for (String _iter804 : struct.pulseIds)
+          {
+            oprot.writeString(_iter804);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBNodes struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.pulseIds = new ArrayList<String>(_list805.size);
+          String _elem806;
+          for (int _i807 = 0; _i807 < _list805.size; ++_i807)
+          {
+            _elem806 = iprot.readString();
+            struct.pulseIds.add(_elem806);
+          }
+        }
+        struct.set_pulseIds_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBPulse.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBPulse.java b/storm-client/src/jvm/org/apache/storm/generated/HBPulse.java
new file mode 100644
index 0000000..574857d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBPulse.java
@@ -0,0 +1,522 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBPulse implements org.apache.thrift.TBase<HBPulse, HBPulse._Fields>, java.io.Serializable, Cloneable, Comparable<HBPulse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBPulse");
+
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DETAILS_FIELD_DESC = new org.apache.thrift.protocol.TField("details", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBPulseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBPulseTupleSchemeFactory());
+  }
+
+  private String id; // required
+  private ByteBuffer details; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ID((short)1, "id"),
+    DETAILS((short)2, "details");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // DETAILS
+          return DETAILS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DETAILS, new org.apache.thrift.meta_data.FieldMetaData("details", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBPulse.class, metaDataMap);
+  }
+
+  public HBPulse() {
+  }
+
+  public HBPulse(
+    String id,
+    ByteBuffer details)
+  {
+    this();
+    this.id = id;
+    this.details = org.apache.thrift.TBaseHelper.copyBinary(details);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBPulse(HBPulse other) {
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_details()) {
+      this.details = org.apache.thrift.TBaseHelper.copyBinary(other.details);
+    }
+  }
+
+  public HBPulse deepCopy() {
+    return new HBPulse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.id = null;
+    this.details = null;
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public byte[] get_details() {
+    set_details(org.apache.thrift.TBaseHelper.rightSize(details));
+    return details == null ? null : details.array();
+  }
+
+  public ByteBuffer buffer_for_details() {
+    return org.apache.thrift.TBaseHelper.copyBinary(details);
+  }
+
+  public void set_details(byte[] details) {
+    this.details = details == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(details, details.length));
+  }
+
+  public void set_details(ByteBuffer details) {
+    this.details = org.apache.thrift.TBaseHelper.copyBinary(details);
+  }
+
+  public void unset_details() {
+    this.details = null;
+  }
+
+  /** Returns true if field details is set (has been assigned a value) and false otherwise */
+  public boolean is_set_details() {
+    return this.details != null;
+  }
+
+  public void set_details_isSet(boolean value) {
+    if (!value) {
+      this.details = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case DETAILS:
+      if (value == null) {
+        unset_details();
+      } else {
+        set_details((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case DETAILS:
+      return get_details();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case DETAILS:
+      return is_set_details();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBPulse)
+      return this.equals((HBPulse)that);
+    return false;
+  }
+
+  public boolean equals(HBPulse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_details = true && this.is_set_details();
+    boolean that_present_details = true && that.is_set_details();
+    if (this_present_details || that_present_details) {
+      if (!(this_present_details && that_present_details))
+        return false;
+      if (!this.details.equals(that.details))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_id = true && (is_set_id());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
+    boolean present_details = true && (is_set_details());
+    list.add(present_details);
+    if (present_details)
+      list.add(details);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBPulse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_details()).compareTo(other.is_set_details());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_details()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.details, other.details);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBPulse(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("details:");
+    if (this.details == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.details, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBPulseStandardSchemeFactory implements SchemeFactory {
+    public HBPulseStandardScheme getScheme() {
+      return new HBPulseStandardScheme();
+    }
+  }
+
+  private static class HBPulseStandardScheme extends StandardScheme<HBPulse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBPulse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.id = iprot.readString();
+              struct.set_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DETAILS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.details = iprot.readBinary();
+              struct.set_details_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBPulse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(struct.id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.details != null) {
+        oprot.writeFieldBegin(DETAILS_FIELD_DESC);
+        oprot.writeBinary(struct.details);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBPulseTupleSchemeFactory implements SchemeFactory {
+    public HBPulseTupleScheme getScheme() {
+      return new HBPulseTupleScheme();
+    }
+  }
+
+  private static class HBPulseTupleScheme extends TupleScheme<HBPulse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBPulse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.id);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_details()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_details()) {
+        oprot.writeBinary(struct.details);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBPulse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.id = iprot.readString();
+      struct.set_id_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.details = iprot.readBinary();
+        struct.set_details_isSet(true);
+      }
+    }
+  }
+
+}
+


[28/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ErrorInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ErrorInfo.java b/storm-client/src/jvm/org/apache/storm/generated/ErrorInfo.java
new file mode 100644
index 0000000..2ab536d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ErrorInfo.java
@@ -0,0 +1,714 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ErrorInfo implements org.apache.thrift.TBase<ErrorInfo, ErrorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<ErrorInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ErrorInfo");
+
+  private static final org.apache.thrift.protocol.TField ERROR_FIELD_DESC = new org.apache.thrift.protocol.TField("error", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField ERROR_TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("error_time_secs", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ErrorInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ErrorInfoTupleSchemeFactory());
+  }
+
+  private String error; // required
+  private int error_time_secs; // required
+  private String host; // optional
+  private int port; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ERROR((short)1, "error"),
+    ERROR_TIME_SECS((short)2, "error_time_secs"),
+    HOST((short)3, "host"),
+    PORT((short)4, "port");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ERROR
+          return ERROR;
+        case 2: // ERROR_TIME_SECS
+          return ERROR_TIME_SECS;
+        case 3: // HOST
+          return HOST;
+        case 4: // PORT
+          return PORT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ERROR_TIME_SECS_ISSET_ID = 0;
+  private static final int __PORT_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.HOST,_Fields.PORT};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ERROR, new org.apache.thrift.meta_data.FieldMetaData("error", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ERROR_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("error_time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ErrorInfo.class, metaDataMap);
+  }
+
+  public ErrorInfo() {
+  }
+
+  public ErrorInfo(
+    String error,
+    int error_time_secs)
+  {
+    this();
+    this.error = error;
+    this.error_time_secs = error_time_secs;
+    set_error_time_secs_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ErrorInfo(ErrorInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_error()) {
+      this.error = other.error;
+    }
+    this.error_time_secs = other.error_time_secs;
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.port = other.port;
+  }
+
+  public ErrorInfo deepCopy() {
+    return new ErrorInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.error = null;
+    set_error_time_secs_isSet(false);
+    this.error_time_secs = 0;
+    this.host = null;
+    set_port_isSet(false);
+    this.port = 0;
+  }
+
+  public String get_error() {
+    return this.error;
+  }
+
+  public void set_error(String error) {
+    this.error = error;
+  }
+
+  public void unset_error() {
+    this.error = null;
+  }
+
+  /** Returns true if field error is set (has been assigned a value) and false otherwise */
+  public boolean is_set_error() {
+    return this.error != null;
+  }
+
+  public void set_error_isSet(boolean value) {
+    if (!value) {
+      this.error = null;
+    }
+  }
+
+  public int get_error_time_secs() {
+    return this.error_time_secs;
+  }
+
+  public void set_error_time_secs(int error_time_secs) {
+    this.error_time_secs = error_time_secs;
+    set_error_time_secs_isSet(true);
+  }
+
+  public void unset_error_time_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERROR_TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field error_time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_error_time_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __ERROR_TIME_SECS_ISSET_ID);
+  }
+
+  public void set_error_time_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERROR_TIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ERROR:
+      if (value == null) {
+        unset_error();
+      } else {
+        set_error((String)value);
+      }
+      break;
+
+    case ERROR_TIME_SECS:
+      if (value == null) {
+        unset_error_time_secs();
+      } else {
+        set_error_time_secs((Integer)value);
+      }
+      break;
+
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ERROR:
+      return get_error();
+
+    case ERROR_TIME_SECS:
+      return get_error_time_secs();
+
+    case HOST:
+      return get_host();
+
+    case PORT:
+      return get_port();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ERROR:
+      return is_set_error();
+    case ERROR_TIME_SECS:
+      return is_set_error_time_secs();
+    case HOST:
+      return is_set_host();
+    case PORT:
+      return is_set_port();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ErrorInfo)
+      return this.equals((ErrorInfo)that);
+    return false;
+  }
+
+  public boolean equals(ErrorInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_error = true && this.is_set_error();
+    boolean that_present_error = true && that.is_set_error();
+    if (this_present_error || that_present_error) {
+      if (!(this_present_error && that_present_error))
+        return false;
+      if (!this.error.equals(that.error))
+        return false;
+    }
+
+    boolean this_present_error_time_secs = true;
+    boolean that_present_error_time_secs = true;
+    if (this_present_error_time_secs || that_present_error_time_secs) {
+      if (!(this_present_error_time_secs && that_present_error_time_secs))
+        return false;
+      if (this.error_time_secs != that.error_time_secs)
+        return false;
+    }
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_port = true && this.is_set_port();
+    boolean that_present_port = true && that.is_set_port();
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_error = true && (is_set_error());
+    list.add(present_error);
+    if (present_error)
+      list.add(error);
+
+    boolean present_error_time_secs = true;
+    list.add(present_error_time_secs);
+    if (present_error_time_secs)
+      list.add(error_time_secs);
+
+    boolean present_host = true && (is_set_host());
+    list.add(present_host);
+    if (present_host)
+      list.add(host);
+
+    boolean present_port = true && (is_set_port());
+    list.add(present_port);
+    if (present_port)
+      list.add(port);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ErrorInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_error()).compareTo(other.is_set_error());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_error()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.error, other.error);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_error_time_secs()).compareTo(other.is_set_error_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_error_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.error_time_secs, other.error_time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ErrorInfo(");
+    boolean first = true;
+
+    sb.append("error:");
+    if (this.error == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.error);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("error_time_secs:");
+    sb.append(this.error_time_secs);
+    first = false;
+    if (is_set_host()) {
+      if (!first) sb.append(", ");
+      sb.append("host:");
+      if (this.host == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.host);
+      }
+      first = false;
+    }
+    if (is_set_port()) {
+      if (!first) sb.append(", ");
+      sb.append("port:");
+      sb.append(this.port);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_error()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'error' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_error_time_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'error_time_secs' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ErrorInfoStandardSchemeFactory implements SchemeFactory {
+    public ErrorInfoStandardScheme getScheme() {
+      return new ErrorInfoStandardScheme();
+    }
+  }
+
+  private static class ErrorInfoStandardScheme extends StandardScheme<ErrorInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ErrorInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ERROR
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.error = iprot.readString();
+              struct.set_error_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // ERROR_TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.error_time_secs = iprot.readI32();
+              struct.set_error_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.host = iprot.readString();
+              struct.set_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.port = iprot.readI32();
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ErrorInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.error != null) {
+        oprot.writeFieldBegin(ERROR_FIELD_DESC);
+        oprot.writeString(struct.error);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(ERROR_TIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.error_time_secs);
+      oprot.writeFieldEnd();
+      if (struct.host != null) {
+        if (struct.is_set_host()) {
+          oprot.writeFieldBegin(HOST_FIELD_DESC);
+          oprot.writeString(struct.host);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_port()) {
+        oprot.writeFieldBegin(PORT_FIELD_DESC);
+        oprot.writeI32(struct.port);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ErrorInfoTupleSchemeFactory implements SchemeFactory {
+    public ErrorInfoTupleScheme getScheme() {
+      return new ErrorInfoTupleScheme();
+    }
+  }
+
+  private static class ErrorInfoTupleScheme extends TupleScheme<ErrorInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ErrorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.error);
+      oprot.writeI32(struct.error_time_secs);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_host()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_port()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_host()) {
+        oprot.writeString(struct.host);
+      }
+      if (struct.is_set_port()) {
+        oprot.writeI32(struct.port);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ErrorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.error = iprot.readString();
+      struct.set_error_isSet(true);
+      struct.error_time_secs = iprot.readI32();
+      struct.set_error_time_secs_isSet(true);
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.host = iprot.readString();
+        struct.set_host_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.port = iprot.readI32();
+        struct.set_port_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ExecutorAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ExecutorAggregateStats.java b/storm-client/src/jvm/org/apache/storm/generated/ExecutorAggregateStats.java
new file mode 100644
index 0000000..5bc85e0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ExecutorAggregateStats.java
@@ -0,0 +1,526 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ExecutorAggregateStats implements org.apache.thrift.TBase<ExecutorAggregateStats, ExecutorAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorAggregateStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorAggregateStats");
+
+  private static final org.apache.thrift.protocol.TField EXEC_SUMMARY_FIELD_DESC = new org.apache.thrift.protocol.TField("exec_summary", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("stats", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ExecutorAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ExecutorAggregateStatsTupleSchemeFactory());
+  }
+
+  private ExecutorSummary exec_summary; // optional
+  private ComponentAggregateStats stats; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXEC_SUMMARY((short)1, "exec_summary"),
+    STATS((short)2, "stats");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXEC_SUMMARY
+          return EXEC_SUMMARY;
+        case 2: // STATS
+          return STATS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.EXEC_SUMMARY,_Fields.STATS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXEC_SUMMARY, new org.apache.thrift.meta_data.FieldMetaData("exec_summary", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSummary.class)));
+    tmpMap.put(_Fields.STATS, new org.apache.thrift.meta_data.FieldMetaData("stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorAggregateStats.class, metaDataMap);
+  }
+
+  public ExecutorAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ExecutorAggregateStats(ExecutorAggregateStats other) {
+    if (other.is_set_exec_summary()) {
+      this.exec_summary = new ExecutorSummary(other.exec_summary);
+    }
+    if (other.is_set_stats()) {
+      this.stats = new ComponentAggregateStats(other.stats);
+    }
+  }
+
+  public ExecutorAggregateStats deepCopy() {
+    return new ExecutorAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.exec_summary = null;
+    this.stats = null;
+  }
+
+  public ExecutorSummary get_exec_summary() {
+    return this.exec_summary;
+  }
+
+  public void set_exec_summary(ExecutorSummary exec_summary) {
+    this.exec_summary = exec_summary;
+  }
+
+  public void unset_exec_summary() {
+    this.exec_summary = null;
+  }
+
+  /** Returns true if field exec_summary is set (has been assigned a value) and false otherwise */
+  public boolean is_set_exec_summary() {
+    return this.exec_summary != null;
+  }
+
+  public void set_exec_summary_isSet(boolean value) {
+    if (!value) {
+      this.exec_summary = null;
+    }
+  }
+
+  public ComponentAggregateStats get_stats() {
+    return this.stats;
+  }
+
+  public void set_stats(ComponentAggregateStats stats) {
+    this.stats = stats;
+  }
+
+  public void unset_stats() {
+    this.stats = null;
+  }
+
+  /** Returns true if field stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_stats() {
+    return this.stats != null;
+  }
+
+  public void set_stats_isSet(boolean value) {
+    if (!value) {
+      this.stats = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXEC_SUMMARY:
+      if (value == null) {
+        unset_exec_summary();
+      } else {
+        set_exec_summary((ExecutorSummary)value);
+      }
+      break;
+
+    case STATS:
+      if (value == null) {
+        unset_stats();
+      } else {
+        set_stats((ComponentAggregateStats)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXEC_SUMMARY:
+      return get_exec_summary();
+
+    case STATS:
+      return get_stats();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXEC_SUMMARY:
+      return is_set_exec_summary();
+    case STATS:
+      return is_set_stats();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ExecutorAggregateStats)
+      return this.equals((ExecutorAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(ExecutorAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_exec_summary = true && this.is_set_exec_summary();
+    boolean that_present_exec_summary = true && that.is_set_exec_summary();
+    if (this_present_exec_summary || that_present_exec_summary) {
+      if (!(this_present_exec_summary && that_present_exec_summary))
+        return false;
+      if (!this.exec_summary.equals(that.exec_summary))
+        return false;
+    }
+
+    boolean this_present_stats = true && this.is_set_stats();
+    boolean that_present_stats = true && that.is_set_stats();
+    if (this_present_stats || that_present_stats) {
+      if (!(this_present_stats && that_present_stats))
+        return false;
+      if (!this.stats.equals(that.stats))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_exec_summary = true && (is_set_exec_summary());
+    list.add(present_exec_summary);
+    if (present_exec_summary)
+      list.add(exec_summary);
+
+    boolean present_stats = true && (is_set_stats());
+    list.add(present_stats);
+    if (present_stats)
+      list.add(stats);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ExecutorAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_exec_summary()).compareTo(other.is_set_exec_summary());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_exec_summary()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exec_summary, other.exec_summary);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_stats()).compareTo(other.is_set_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, other.stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ExecutorAggregateStats(");
+    boolean first = true;
+
+    if (is_set_exec_summary()) {
+      sb.append("exec_summary:");
+      if (this.exec_summary == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.exec_summary);
+      }
+      first = false;
+    }
+    if (is_set_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("stats:");
+      if (this.stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.stats);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (exec_summary != null) {
+      exec_summary.validate();
+    }
+    if (stats != null) {
+      stats.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ExecutorAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public ExecutorAggregateStatsStandardScheme getScheme() {
+      return new ExecutorAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class ExecutorAggregateStatsStandardScheme extends StandardScheme<ExecutorAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXEC_SUMMARY
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.exec_summary = new ExecutorSummary();
+              struct.exec_summary.read(iprot);
+              struct.set_exec_summary_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.stats = new ComponentAggregateStats();
+              struct.stats.read(iprot);
+              struct.set_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.exec_summary != null) {
+        if (struct.is_set_exec_summary()) {
+          oprot.writeFieldBegin(EXEC_SUMMARY_FIELD_DESC);
+          struct.exec_summary.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.stats != null) {
+        if (struct.is_set_stats()) {
+          oprot.writeFieldBegin(STATS_FIELD_DESC);
+          struct.stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ExecutorAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public ExecutorAggregateStatsTupleScheme getScheme() {
+      return new ExecutorAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class ExecutorAggregateStatsTupleScheme extends TupleScheme<ExecutorAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_exec_summary()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_stats()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_exec_summary()) {
+        struct.exec_summary.write(oprot);
+      }
+      if (struct.is_set_stats()) {
+        struct.stats.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.exec_summary = new ExecutorSummary();
+        struct.exec_summary.read(iprot);
+        struct.set_exec_summary_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.stats = new ComponentAggregateStats();
+        struct.stats.read(iprot);
+        struct.set_stats_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ExecutorInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ExecutorInfo.java b/storm-client/src/jvm/org/apache/storm/generated/ExecutorInfo.java
new file mode 100644
index 0000000..b3d7bab
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ExecutorInfo.java
@@ -0,0 +1,499 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ExecutorInfo implements org.apache.thrift.TBase<ExecutorInfo, ExecutorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorInfo");
+
+  private static final org.apache.thrift.protocol.TField TASK_START_FIELD_DESC = new org.apache.thrift.protocol.TField("task_start", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField TASK_END_FIELD_DESC = new org.apache.thrift.protocol.TField("task_end", org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ExecutorInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ExecutorInfoTupleSchemeFactory());
+  }
+
+  private int task_start; // required
+  private int task_end; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TASK_START((short)1, "task_start"),
+    TASK_END((short)2, "task_end");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TASK_START
+          return TASK_START;
+        case 2: // TASK_END
+          return TASK_END;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TASK_START_ISSET_ID = 0;
+  private static final int __TASK_END_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TASK_START, new org.apache.thrift.meta_data.FieldMetaData("task_start", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.TASK_END, new org.apache.thrift.meta_data.FieldMetaData("task_end", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorInfo.class, metaDataMap);
+  }
+
+  public ExecutorInfo() {
+  }
+
+  public ExecutorInfo(
+    int task_start,
+    int task_end)
+  {
+    this();
+    this.task_start = task_start;
+    set_task_start_isSet(true);
+    this.task_end = task_end;
+    set_task_end_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ExecutorInfo(ExecutorInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.task_start = other.task_start;
+    this.task_end = other.task_end;
+  }
+
+  public ExecutorInfo deepCopy() {
+    return new ExecutorInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    set_task_start_isSet(false);
+    this.task_start = 0;
+    set_task_end_isSet(false);
+    this.task_end = 0;
+  }
+
+  public int get_task_start() {
+    return this.task_start;
+  }
+
+  public void set_task_start(int task_start) {
+    this.task_start = task_start;
+    set_task_start_isSet(true);
+  }
+
+  public void unset_task_start() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TASK_START_ISSET_ID);
+  }
+
+  /** Returns true if field task_start is set (has been assigned a value) and false otherwise */
+  public boolean is_set_task_start() {
+    return EncodingUtils.testBit(__isset_bitfield, __TASK_START_ISSET_ID);
+  }
+
+  public void set_task_start_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TASK_START_ISSET_ID, value);
+  }
+
+  public int get_task_end() {
+    return this.task_end;
+  }
+
+  public void set_task_end(int task_end) {
+    this.task_end = task_end;
+    set_task_end_isSet(true);
+  }
+
+  public void unset_task_end() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TASK_END_ISSET_ID);
+  }
+
+  /** Returns true if field task_end is set (has been assigned a value) and false otherwise */
+  public boolean is_set_task_end() {
+    return EncodingUtils.testBit(__isset_bitfield, __TASK_END_ISSET_ID);
+  }
+
+  public void set_task_end_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TASK_END_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TASK_START:
+      if (value == null) {
+        unset_task_start();
+      } else {
+        set_task_start((Integer)value);
+      }
+      break;
+
+    case TASK_END:
+      if (value == null) {
+        unset_task_end();
+      } else {
+        set_task_end((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TASK_START:
+      return get_task_start();
+
+    case TASK_END:
+      return get_task_end();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TASK_START:
+      return is_set_task_start();
+    case TASK_END:
+      return is_set_task_end();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ExecutorInfo)
+      return this.equals((ExecutorInfo)that);
+    return false;
+  }
+
+  public boolean equals(ExecutorInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_task_start = true;
+    boolean that_present_task_start = true;
+    if (this_present_task_start || that_present_task_start) {
+      if (!(this_present_task_start && that_present_task_start))
+        return false;
+      if (this.task_start != that.task_start)
+        return false;
+    }
+
+    boolean this_present_task_end = true;
+    boolean that_present_task_end = true;
+    if (this_present_task_end || that_present_task_end) {
+      if (!(this_present_task_end && that_present_task_end))
+        return false;
+      if (this.task_end != that.task_end)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_task_start = true;
+    list.add(present_task_start);
+    if (present_task_start)
+      list.add(task_start);
+
+    boolean present_task_end = true;
+    list.add(present_task_end);
+    if (present_task_end)
+      list.add(task_end);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ExecutorInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_task_start()).compareTo(other.is_set_task_start());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_task_start()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.task_start, other.task_start);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_task_end()).compareTo(other.is_set_task_end());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_task_end()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.task_end, other.task_end);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ExecutorInfo(");
+    boolean first = true;
+
+    sb.append("task_start:");
+    sb.append(this.task_start);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("task_end:");
+    sb.append(this.task_end);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_task_start()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'task_start' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_task_end()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'task_end' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ExecutorInfoStandardSchemeFactory implements SchemeFactory {
+    public ExecutorInfoStandardScheme getScheme() {
+      return new ExecutorInfoStandardScheme();
+    }
+  }
+
+  private static class ExecutorInfoStandardScheme extends StandardScheme<ExecutorInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TASK_START
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.task_start = iprot.readI32();
+              struct.set_task_start_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TASK_END
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.task_end = iprot.readI32();
+              struct.set_task_end_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(TASK_START_FIELD_DESC);
+      oprot.writeI32(struct.task_start);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(TASK_END_FIELD_DESC);
+      oprot.writeI32(struct.task_end);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ExecutorInfoTupleSchemeFactory implements SchemeFactory {
+    public ExecutorInfoTupleScheme getScheme() {
+      return new ExecutorInfoTupleScheme();
+    }
+  }
+
+  private static class ExecutorInfoTupleScheme extends TupleScheme<ExecutorInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.task_start);
+      oprot.writeI32(struct.task_end);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.task_start = iprot.readI32();
+      struct.set_task_start_isSet(true);
+      struct.task_end = iprot.readI32();
+      struct.set_task_end_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ExecutorSpecificStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ExecutorSpecificStats.java b/storm-client/src/jvm/org/apache/storm/generated/ExecutorSpecificStats.java
new file mode 100644
index 0000000..d91f197
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ExecutorSpecificStats.java
@@ -0,0 +1,387 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class ExecutorSpecificStats extends org.apache.thrift.TUnion<ExecutorSpecificStats, ExecutorSpecificStats._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorSpecificStats");
+  private static final org.apache.thrift.protocol.TField BOLT_FIELD_DESC = new org.apache.thrift.protocol.TField("bolt", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField SPOUT_FIELD_DESC = new org.apache.thrift.protocol.TField("spout", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    BOLT((short)1, "bolt"),
+    SPOUT((short)2, "spout");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // BOLT
+          return BOLT;
+        case 2: // SPOUT
+          return SPOUT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.BOLT, new org.apache.thrift.meta_data.FieldMetaData("bolt", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, BoltStats.class)));
+    tmpMap.put(_Fields.SPOUT, new org.apache.thrift.meta_data.FieldMetaData("spout", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpoutStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorSpecificStats.class, metaDataMap);
+  }
+
+  public ExecutorSpecificStats() {
+    super();
+  }
+
+  public ExecutorSpecificStats(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public ExecutorSpecificStats(ExecutorSpecificStats other) {
+    super(other);
+  }
+  public ExecutorSpecificStats deepCopy() {
+    return new ExecutorSpecificStats(this);
+  }
+
+  public static ExecutorSpecificStats bolt(BoltStats value) {
+    ExecutorSpecificStats x = new ExecutorSpecificStats();
+    x.set_bolt(value);
+    return x;
+  }
+
+  public static ExecutorSpecificStats spout(SpoutStats value) {
+    ExecutorSpecificStats x = new ExecutorSpecificStats();
+    x.set_spout(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case BOLT:
+        if (value instanceof BoltStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type BoltStats for field 'bolt', but got " + value.getClass().getSimpleName());
+      case SPOUT:
+        if (value instanceof SpoutStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type SpoutStats for field 'spout', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case BOLT:
+          if (field.type == BOLT_FIELD_DESC.type) {
+            BoltStats bolt;
+            bolt = new BoltStats();
+            bolt.read(iprot);
+            return bolt;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SPOUT:
+          if (field.type == SPOUT_FIELD_DESC.type) {
+            SpoutStats spout;
+            spout = new SpoutStats();
+            spout.read(iprot);
+            return spout;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case BOLT:
+        BoltStats bolt = (BoltStats)value_;
+        bolt.write(oprot);
+        return;
+      case SPOUT:
+        SpoutStats spout = (SpoutStats)value_;
+        spout.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case BOLT:
+          BoltStats bolt;
+          bolt = new BoltStats();
+          bolt.read(iprot);
+          return bolt;
+        case SPOUT:
+          SpoutStats spout;
+          spout = new SpoutStats();
+          spout.read(iprot);
+          return spout;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case BOLT:
+        BoltStats bolt = (BoltStats)value_;
+        bolt.write(oprot);
+        return;
+      case SPOUT:
+        SpoutStats spout = (SpoutStats)value_;
+        spout.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case BOLT:
+        return BOLT_FIELD_DESC;
+      case SPOUT:
+        return SPOUT_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public BoltStats get_bolt() {
+    if (getSetField() == _Fields.BOLT) {
+      return (BoltStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'bolt' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_bolt(BoltStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.BOLT;
+    value_ = value;
+  }
+
+  public SpoutStats get_spout() {
+    if (getSetField() == _Fields.SPOUT) {
+      return (SpoutStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'spout' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_spout(SpoutStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SPOUT;
+    value_ = value;
+  }
+
+  public boolean is_set_bolt() {
+    return setField_ == _Fields.BOLT;
+  }
+
+
+  public boolean is_set_spout() {
+    return setField_ == _Fields.SPOUT;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof ExecutorSpecificStats) {
+      return equals((ExecutorSpecificStats)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(ExecutorSpecificStats other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(ExecutorSpecificStats other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}


[42/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/DevicesCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/DevicesCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/DevicesCore.java
new file mode 100755
index 0000000..c38f5fe
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/DevicesCore.java
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+import org.apache.storm.container.cgroup.Device;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+public class DevicesCore implements CgroupCore {
+
+    private final String dir;
+
+    private static final String DEVICES_ALLOW = "/devices.allow";
+    private static final String DEVICES_DENY = "/devices.deny";
+    private static final String DEVICES_LIST = "/devices.list";
+
+    private static final char TYPE_ALL = 'a';
+    private static final char TYPE_BLOCK = 'b';
+    private static final char TYPE_CHAR = 'c';
+
+    private static final int ACCESS_READ = 1;
+    private static final int ACCESS_WRITE = 2;
+    private static final int ACCESS_CREATE = 4;
+
+    private static final char ACCESS_READ_CH = 'r';
+    private static final char ACCESS_WRITE_CH = 'w';
+    private static final char ACCESS_CREATE_CH = 'm';
+
+    private static final Logger LOG = LoggerFactory.getLogger(DevicesCore.class);
+
+    public DevicesCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.devices;
+    }
+
+    public static class Record {
+        Device device;
+        char type;
+        int accesses;
+
+        public Record(char type, Device device, int accesses) {
+            this.type = type;
+            this.device = device;
+            this.accesses = accesses;
+        }
+
+        public Record(String output) {
+            if (output.contains("*")) {
+                LOG.debug("Pre: {}", output);
+                output = output.replaceAll("\\*", "-1");
+                LOG.debug("After: {}",output);
+            }
+            String[] splits = output.split("[: ]");
+            type = splits[0].charAt(0);
+            int major = Integer.parseInt(splits[1]);
+            int minor = Integer.parseInt(splits[2]);
+            device = new Device(major, minor);
+            accesses = 0;
+            for (char c : splits[3].toCharArray()) {
+                if (c == ACCESS_READ_CH) {
+                    accesses |= ACCESS_READ;
+                }
+                if (c == ACCESS_CREATE_CH) {
+                    accesses |= ACCESS_CREATE;
+                }
+                if (c == ACCESS_WRITE_CH) {
+                    accesses |= ACCESS_WRITE;
+                }
+            }
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append(type);
+            sb.append(' ');
+            sb.append(device.major);
+            sb.append(':');
+            sb.append(device.minor);
+            sb.append(' ');
+            sb.append(getAccessesFlag(accesses));
+
+            return sb.toString();
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + accesses;
+            result = prime * result + ((device == null) ? 0 : device.hashCode());
+            result = prime * result + type;
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) {
+                return true;
+            }
+            if (obj == null) {
+                return false;
+            }
+            if (getClass() != obj.getClass()) {
+                return false;
+            }
+            Record other = (Record) obj;
+            if (accesses != other.accesses) {
+                return false;
+            }
+            if (device == null) {
+                if (other.device != null) {
+                    return false;
+                }
+            } else if (!device.equals(other.device)) {
+                return false;
+            }
+            if (type != other.type) {
+                return false;
+            }
+            return true;
+        }
+
+        public static Record[] parseRecordList(List<String> output) {
+            Record[] records = new Record[output.size()];
+            for (int i = 0, l = output.size(); i < l; i++) {
+                records[i] = new Record(output.get(i));
+            }
+
+            return records;
+        }
+
+        public static StringBuilder getAccessesFlag(int accesses) {
+            StringBuilder sb = new StringBuilder();
+            if ((accesses & ACCESS_READ) != 0) {
+                sb.append(ACCESS_READ_CH);
+            }
+            if ((accesses & ACCESS_WRITE) != 0) {
+                sb.append(ACCESS_WRITE_CH);
+            }
+            if ((accesses & ACCESS_CREATE) != 0) {
+                sb.append(ACCESS_CREATE_CH);
+            }
+            return sb;
+        }
+    }
+
+    private void setPermission(String prop, char type, Device device, int accesses) throws IOException {
+        Record record = new Record(type, device, accesses);
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, prop), record.toString());
+    }
+
+    public void setAllow(char type, Device device, int accesses) throws IOException {
+        setPermission(DEVICES_ALLOW, type, device, accesses);
+    }
+
+    public void setDeny(char type, Device device, int accesses) throws IOException {
+        setPermission(DEVICES_DENY, type, device, accesses);
+    }
+
+    public Record[] getList() throws IOException {
+        List<String> output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, DEVICES_LIST));
+        return Record.parseRecordList(output);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/FreezerCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/FreezerCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/FreezerCore.java
new file mode 100755
index 0000000..89e13dd
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/FreezerCore.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+
+import java.io.IOException;
+
+public class FreezerCore implements CgroupCore {
+
+    public static final String FREEZER_STATE = "/freezer.state";
+
+    private final String dir;
+
+    public FreezerCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.freezer;
+    }
+
+    public void setState(State state) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, FREEZER_STATE), state.name().toUpperCase());
+    }
+
+    public State getState() throws IOException {
+        return State.getStateValue(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, FREEZER_STATE)).get(0));
+    }
+
+    public enum State {
+        frozen, freezing, thawed;
+
+        public static State getStateValue(String state) {
+            if (state.equals("FROZEN")) {
+                return frozen;
+            }
+            else if (state.equals("FREEZING")) {
+                return freezing;
+            }
+            else if (state.equals("THAWED")) {
+                return thawed;
+            }
+            else {
+                return null;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/MemoryCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/MemoryCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/MemoryCore.java
new file mode 100755
index 0000000..9bd6a72
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/MemoryCore.java
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+
+import java.io.IOException;
+
+public class MemoryCore implements CgroupCore {
+
+    public static final String MEMORY_STAT = "/memory.stat";
+    public static final String MEMORY_USAGE_IN_BYTES = "/memory.usage_in_bytes";
+    public static final String MEMORY_MEMSW_USAGE_IN_BYTES = "/memory.memsw.usage_in_bytes";
+    public static final String MEMORY_MAX_USAGE_IN_BYTES = "/memory.max_usage_in_bytes";
+    public static final String MEMORY_MEMSW_MAX_USAGE_IN_BYTES = "/memory.memsw.max_usage_in_bytes";
+    public static final String MEMORY_LIMIT_IN_BYTES = "/memory.limit_in_bytes";
+    public static final String MEMORY_MEMSW_LIMIT_IN_BYTES = "/memory.memsw.limit_in_bytes";
+    public static final String MEMORY_FAILCNT = "/memory.failcnt";
+    public static final String MEMORY_MEMSW_FAILCNT = "/memory.memsw.failcnt";
+    public static final String MEMORY_FORCE_EMPTY = "/memory.force_empty";
+    public static final String MEMORY_SWAPPINESS = "/memory.swappiness";
+    public static final String MEMORY_USE_HIERARCHY = "/memory.use_hierarchy";
+    public static final String MEMORY_OOM_CONTROL = "/memory.oom_control";
+
+    private final String dir;
+
+    public MemoryCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.memory;
+    }
+
+    public static class Stat {
+        public final long cacheSize;
+        public final long rssSize;
+        public final long mappedFileSize;
+        public final long pgpginNum;
+        public final long pgpgoutNum;
+        public final long swapSize;
+        public final long activeAnonSize;
+        public final long inactiveAnonSize;
+        public final long activeFileSize;
+        public final long inactiveFileSize;
+        public final long unevictableSize;
+        public final long hierarchicalMemoryLimitSize;
+        public final long hierarchicalMemSwapLimitSize;
+        public final long totalCacheSize;
+        public final long totalRssSize;
+        public final long totalMappedFileSize;
+        public final long totalPgpginNum;
+        public final long totalPgpgoutNum;
+        public final long totalSwapSize;
+        public final long totalActiveAnonSize;
+        public final long totalInactiveAnonSize;
+        public final long totalActiveFileSize;
+        public final long totalInactiveFileSize;
+        public final long totalUnevictableSize;
+        public final long totalHierarchicalMemoryLimitSize;
+        public final long totalHierarchicalMemSwapLimitSize;
+
+        public Stat(String output) {
+            String[] splits = output.split("\n");
+            this.cacheSize = Long.parseLong(splits[0]);
+            this.rssSize = Long.parseLong(splits[1]);
+            this.mappedFileSize = Long.parseLong(splits[2]);
+            this.pgpginNum = Long.parseLong(splits[3]);
+            this.pgpgoutNum = Long.parseLong(splits[4]);
+            this.swapSize = Long.parseLong(splits[5]);
+            this.inactiveAnonSize = Long.parseLong(splits[6]);
+            this.activeAnonSize = Long.parseLong(splits[7]);
+            this.inactiveFileSize = Long.parseLong(splits[8]);
+            this.activeFileSize = Long.parseLong(splits[9]);
+            this.unevictableSize = Long.parseLong(splits[10]);
+            this.hierarchicalMemoryLimitSize = Long.parseLong(splits[11]);
+            this.hierarchicalMemSwapLimitSize = Long.parseLong(splits[12]);
+            this.totalCacheSize = Long.parseLong(splits[13]);
+            this.totalRssSize = Long.parseLong(splits[14]);
+            this.totalMappedFileSize = Long.parseLong(splits[15]);
+            this.totalPgpginNum = Long.parseLong(splits[16]);
+            this.totalPgpgoutNum = Long.parseLong(splits[17]);
+            this.totalSwapSize = Long.parseLong(splits[18]);
+            this.totalInactiveAnonSize = Long.parseLong(splits[19]);
+            this.totalActiveAnonSize = Long.parseLong(splits[20]);
+            this.totalInactiveFileSize = Long.parseLong(splits[21]);
+            this.totalActiveFileSize = Long.parseLong(splits[22]);
+            this.totalUnevictableSize = Long.parseLong(splits[23]);
+            this.totalHierarchicalMemoryLimitSize = Long.parseLong(splits[24]);
+            this.totalHierarchicalMemSwapLimitSize = Long.parseLong(splits[25]);
+        }
+    }
+
+    public Stat getStat() throws IOException {
+        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_STAT)).get(0);
+        Stat stat = new Stat(output);
+        return stat;
+    }
+
+    public long getPhysicalUsage() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_USAGE_IN_BYTES)).get(0));
+    }
+
+    public long getWithSwapUsage() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_MEMSW_USAGE_IN_BYTES)).get(0));
+    }
+
+    public long getMaxPhysicalUsage() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_MAX_USAGE_IN_BYTES)).get(0));
+    }
+
+    public long getMaxWithSwapUsage() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_MEMSW_MAX_USAGE_IN_BYTES)).get(0));
+    }
+
+    public void setPhysicalUsageLimit(long value) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, MEMORY_LIMIT_IN_BYTES), String.valueOf(value));
+    }
+
+    public long getPhysicalUsageLimit() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_LIMIT_IN_BYTES)).get(0));
+    }
+
+    public void setWithSwapUsageLimit(long value) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, MEMORY_MEMSW_LIMIT_IN_BYTES), String.valueOf(value));
+    }
+
+    public long getWithSwapUsageLimit() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_MEMSW_LIMIT_IN_BYTES)).get(0));
+    }
+
+    public int getPhysicalFailCount() throws IOException {
+        return Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_FAILCNT)).get(0));
+    }
+
+    public int getWithSwapFailCount() throws IOException {
+        return Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_MEMSW_FAILCNT)).get(0));
+    }
+
+    public void clearForceEmpty() throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, MEMORY_FORCE_EMPTY), String.valueOf(0));
+    }
+
+    public void setSwappiness(int value) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, MEMORY_SWAPPINESS), String.valueOf(value));
+    }
+
+    public int getSwappiness() throws IOException {
+        return Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_SWAPPINESS)).get(0));
+    }
+
+    public void setUseHierarchy(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, MEMORY_USE_HIERARCHY), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isUseHierarchy() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_USE_HIERARCHY)).get(0));
+        return output > 0;
+    }
+
+    public void setOomControl(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, MEMORY_OOM_CONTROL), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isOomControl() throws IOException {
+        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, MEMORY_OOM_CONTROL)).get(0);
+        output = output.split("\n")[0].split("[\\s]")[1];
+        int value = Integer.parseInt(output);
+        return value > 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetClsCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetClsCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetClsCore.java
new file mode 100755
index 0000000..d3dd5a7
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetClsCore.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+import org.apache.storm.container.cgroup.Device;
+
+import java.io.IOException;
+
+public class NetClsCore implements CgroupCore {
+
+    public static final String NET_CLS_CLASSID = "/net_cls.classid";
+
+    private final String dir;
+
+    public NetClsCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.net_cls;
+    }
+
+    private StringBuilder toHex(int num) {
+        String hex = num + "";
+        StringBuilder sb = new StringBuilder();
+        int l = hex.length();
+        if (l > 4) {
+            hex = hex.substring(l - 4 - 1, l);
+        }
+        for (; l < 4; l++) {
+            sb.append('0');
+        }
+        sb.append(hex);
+        return sb;
+    }
+
+    public void setClassId(int major, int minor) throws IOException {
+        StringBuilder sb = new StringBuilder("0x");
+        sb.append(toHex(major));
+        sb.append(toHex(minor));
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NET_CLS_CLASSID), sb.toString());
+    }
+
+    public Device getClassId() throws IOException {
+        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NET_CLS_CLASSID)).get(0);
+        output = Integer.toHexString(Integer.parseInt(output));
+        int major = Integer.parseInt(output.substring(0, output.length() - 4));
+        int minor = Integer.parseInt(output.substring(output.length() - 4));
+        return new Device(major, minor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetPrioCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetPrioCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetPrioCore.java
new file mode 100755
index 0000000..b83b81a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/NetPrioCore.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class NetPrioCore implements CgroupCore {
+
+    public static final String NET_PRIO_PRIOIDX = "/net_prio.prioidx";
+    public static final String NET_PRIO_IFPRIOMAP = "/net_prio.ifpriomap";
+
+    private final String dir;
+
+    public NetPrioCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.net_prio;
+    }
+
+    public int getPrioId() throws IOException {
+        return Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NET_PRIO_PRIOIDX)).get(0));
+    }
+
+    public void setIfPrioMap(String iface, int priority) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        sb.append(iface);
+        sb.append(' ');
+        sb.append(priority);
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NET_PRIO_IFPRIOMAP), sb.toString());
+    }
+
+    public Map<String, Integer> getIfPrioMap() throws IOException {
+        Map<String, Integer> result = new HashMap<String, Integer>();
+        List<String> strs = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NET_PRIO_IFPRIOMAP));
+        for (String str : strs) {
+            String[] strArgs = str.split(" ");
+            result.put(strArgs[0], Integer.valueOf(strArgs[1]));
+        }
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java
new file mode 100644
index 0000000..2a9a1d8
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.coordination;
+
+import org.apache.storm.coordination.CoordinatedBolt.FinishedCallback;
+import org.apache.storm.coordination.CoordinatedBolt.TimeoutCallback;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.FailedException;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.Utils;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCallback {
+    public static final Logger LOG = LoggerFactory.getLogger(BatchBoltExecutor.class);
+
+    byte[] _boltSer;
+    Map<Object, IBatchBolt> _openTransactions;
+    Map _conf;
+    TopologyContext _context;
+    BatchOutputCollectorImpl _collector;
+    
+    public BatchBoltExecutor(IBatchBolt bolt) {
+        _boltSer = Utils.javaSerialize(bolt);
+    }
+    
+    @Override
+    public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+        _conf = conf;
+        _context = context;
+        _collector = new BatchOutputCollectorImpl(collector);
+        _openTransactions = new HashMap<>();
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        Object id = input.getValue(0);
+        IBatchBolt bolt = getBatchBolt(id);
+        try {
+             bolt.execute(input);
+            _collector.ack(input);
+        } catch(FailedException e) {
+            LOG.error("Failed to process tuple in batch", e);
+            _collector.fail(input);                
+        }
+    }
+
+    @Override
+    public void cleanup() {
+    }
+
+    @Override
+    public void finishedId(Object id) {
+        IBatchBolt bolt = getBatchBolt(id);
+        _openTransactions.remove(id);
+        bolt.finishBatch();
+    }
+
+    @Override
+    public void timeoutId(Object attempt) {
+        _openTransactions.remove(attempt);        
+    }    
+    
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        newTransactionalBolt().declareOutputFields(declarer);
+    }
+    
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return newTransactionalBolt().getComponentConfiguration();
+    }
+    
+    private IBatchBolt getBatchBolt(Object id) {
+        IBatchBolt bolt = _openTransactions.get(id);
+        if(bolt==null) {
+            bolt = newTransactionalBolt();
+            bolt.prepare(_conf, _context, _collector, id);
+            _openTransactions.put(id, bolt);            
+        }
+        return bolt;
+    }
+    
+    private IBatchBolt newTransactionalBolt() {
+        return Utils.javaDeserialize(_boltSer, IBatchBolt.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java
new file mode 100644
index 0000000..e1581eb
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java
@@ -0,0 +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 org.apache.storm.coordination;
+
+import org.apache.storm.utils.Utils;
+import java.util.List;
+
+public abstract class BatchOutputCollector {
+
+    /**
+     * Emits a tuple to the default output stream.
+     */
+    public List<Integer> emit(List<Object> tuple) {
+        return emit(Utils.DEFAULT_STREAM_ID, tuple);
+    }
+
+    public abstract List<Integer> emit(String streamId, List<Object> tuple);
+    
+    /**
+     * Emits a tuple to the specified task on the default output stream. This output
+     * stream must have been declared as a direct stream, and the specified task must
+     * use a direct grouping on this stream to receive the message.
+     */
+    public void emitDirect(int taskId, List<Object> tuple) {
+        emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple);
+    }
+    
+    public abstract void emitDirect(int taskId, String streamId, List<Object> tuple); 
+    
+    public abstract void reportError(Throwable error);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java
new file mode 100644
index 0000000..246ae5d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.coordination;
+
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.tuple.Tuple;
+import java.util.List;
+
+public class BatchOutputCollectorImpl extends BatchOutputCollector {
+    OutputCollector _collector;
+    
+    public BatchOutputCollectorImpl(OutputCollector collector) {
+        _collector = collector;
+    }
+    
+    @Override
+    public List<Integer> emit(String streamId, List<Object> tuple) {
+        return _collector.emit(streamId, tuple);
+    }
+
+    @Override
+    public void emitDirect(int taskId, String streamId, List<Object> tuple) {
+        _collector.emitDirect(taskId, streamId, tuple);
+    }
+
+    @Override
+    public void reportError(Throwable error) {
+        _collector.reportError(error);
+    }
+    
+    public void ack(Tuple tup) {
+        _collector.ack(tup);
+    }
+    
+    public void fail(Tuple tup) {
+        _collector.fail(tup);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/coordination/BatchSubtopologyBuilder.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/BatchSubtopologyBuilder.java b/storm-client/src/jvm/org/apache/storm/coordination/BatchSubtopologyBuilder.java
new file mode 100644
index 0000000..bd72c89
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/coordination/BatchSubtopologyBuilder.java
@@ -0,0 +1,447 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.coordination;
+
+import org.apache.storm.Constants;
+import org.apache.storm.coordination.CoordinatedBolt.SourceArgs;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.grouping.CustomStreamGrouping;
+import org.apache.storm.grouping.PartialKeyGrouping;
+import org.apache.storm.topology.BaseConfigurationDeclarer;
+import org.apache.storm.topology.BasicBoltExecutor;
+import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.InputDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Fields;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class BatchSubtopologyBuilder {
+    Map<String, Component> _bolts = new HashMap<String, Component>();
+    Component _masterBolt;
+    String _masterId;
+    
+    public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt, Number boltParallelism) {
+        Integer p = boltParallelism == null ? null : boltParallelism.intValue();
+        _masterBolt = new Component(new BasicBoltExecutor(masterBolt), p);
+        _masterId = masterBoltId;
+    }
+    
+    public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt) {
+        this(masterBoltId, masterBolt, null);
+    }
+    
+    public BoltDeclarer getMasterDeclarer() {
+        return new BoltDeclarerImpl(_masterBolt);
+    }
+        
+    public BoltDeclarer setBolt(String id, IBatchBolt bolt) {
+        return setBolt(id, bolt, null);
+    }
+    
+    public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number parallelism) {
+        return setBolt(id, new BatchBoltExecutor(bolt), parallelism);
+    }     
+    
+    public BoltDeclarer setBolt(String id, IBasicBolt bolt) {
+        return setBolt(id, bolt, null);
+    }    
+    
+    public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism) {
+        return setBolt(id, new BasicBoltExecutor(bolt), parallelism);
+    }
+    
+    private BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism) {
+        Integer p = null;
+        if(parallelism!=null) p = parallelism.intValue();
+        Component component = new Component(bolt, p);
+        _bolts.put(id, component);
+        return new BoltDeclarerImpl(component);
+    }
+    
+    public void extendTopology(TopologyBuilder builder) {
+        BoltDeclarer declarer = builder.setBolt(_masterId, new CoordinatedBolt(_masterBolt.bolt), _masterBolt.parallelism);
+        for(InputDeclaration decl: _masterBolt.declarations) {
+            decl.declare(declarer);
+        }
+        for(Map conf: _masterBolt.componentConfs) {
+            declarer.addConfigurations(conf);
+        }
+        for(String id: _bolts.keySet()) {
+            Component component = _bolts.get(id);
+            Map<String, SourceArgs> coordinatedArgs = new HashMap<String, SourceArgs>();
+            for(String c: componentBoltSubscriptions(component)) {
+                SourceArgs source;
+                if(c.equals(_masterId)) {
+                    source = SourceArgs.single();
+                } else {
+                    source = SourceArgs.all();
+                }
+                coordinatedArgs.put(c, source);                    
+            }
+            
+
+            BoltDeclarer input = builder.setBolt(id,
+                                                  new CoordinatedBolt(component.bolt,
+                                                                      coordinatedArgs,
+                                                                      null),
+                                                  component.parallelism);
+            for(Map conf: component.componentConfs) {
+                input.addConfigurations(conf);
+            }
+            for(String c: componentBoltSubscriptions(component)) {
+                input.directGrouping(c, Constants.COORDINATED_STREAM_ID);
+            }
+            for(InputDeclaration d: component.declarations) {
+                d.declare(input);
+            }
+        }        
+    }
+        
+    private Set<String> componentBoltSubscriptions(Component component) {
+        Set<String> ret = new HashSet<String>();
+        for(InputDeclaration d: component.declarations) {
+            ret.add(d.getComponent());
+        }
+        return ret;
+    }
+
+    private static class Component {
+        public IRichBolt bolt;
+        public Integer parallelism;
+        public List<InputDeclaration> declarations = new ArrayList<InputDeclaration>();
+        public List<Map<String, Object>> componentConfs = new ArrayList<>();
+        
+        public Component(IRichBolt bolt, Integer parallelism) {
+            this.bolt = bolt;
+            this.parallelism = parallelism;
+        }
+    }
+    
+    private static interface InputDeclaration {
+        void declare(InputDeclarer declarer);
+        String getComponent();
+    }
+        
+    private static class BoltDeclarerImpl extends BaseConfigurationDeclarer<BoltDeclarer> implements BoltDeclarer {
+        Component _component;
+        
+        public BoltDeclarerImpl(Component component) {
+            _component = component;
+        }
+        
+        @Override
+        public BoltDeclarer fieldsGrouping(final String component, final Fields fields) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.fieldsGrouping(component, fields);
+                }
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer fieldsGrouping(final String component, final String streamId, final Fields fields) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.fieldsGrouping(component, streamId, fields);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer globalGrouping(final String component) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.globalGrouping(component);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer globalGrouping(final String component, final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.globalGrouping(component, streamId);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer shuffleGrouping(final String component) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.shuffleGrouping(component);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer shuffleGrouping(final String component, final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.shuffleGrouping(component, streamId);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer localOrShuffleGrouping(final String component) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.localOrShuffleGrouping(component);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer localOrShuffleGrouping(final String component, final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.localOrShuffleGrouping(component, streamId);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+        
+        @Override
+        public BoltDeclarer noneGrouping(final String component) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.noneGrouping(component);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer noneGrouping(final String component, final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.noneGrouping(component, streamId);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer allGrouping(final String component) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.allGrouping(component);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer allGrouping(final String component, final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.allGrouping(component, streamId);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer directGrouping(final String component) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.directGrouping(component);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer directGrouping(final String component, final String streamId) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.directGrouping(component, streamId);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer partialKeyGrouping(String componentId, Fields fields) {
+            return customGrouping(componentId, new PartialKeyGrouping(fields));
+        }
+
+        @Override
+        public BoltDeclarer partialKeyGrouping(String componentId, String streamId, Fields fields) {
+            return customGrouping(componentId, streamId, new PartialKeyGrouping(fields));
+        }
+        
+        @Override
+        public BoltDeclarer customGrouping(final String component, final CustomStreamGrouping grouping) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.customGrouping(component, grouping);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;        
+        }
+
+        @Override
+        public BoltDeclarer customGrouping(final String component, final String streamId, final CustomStreamGrouping grouping) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.customGrouping(component, streamId, grouping);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return component;
+                }                
+            });
+            return this;
+        }
+
+        @Override
+        public BoltDeclarer grouping(final GlobalStreamId stream, final Grouping grouping) {
+            addDeclaration(new InputDeclaration() {
+                @Override
+                public void declare(InputDeclarer declarer) {
+                    declarer.grouping(stream, grouping);
+                }                
+
+                @Override
+                public String getComponent() {
+                    return stream.get_componentId();
+                }                
+            });
+            return this;
+        }
+        
+        private void addDeclaration(InputDeclaration declaration) {
+            _component.declarations.add(declaration);
+        }
+
+        @Override
+        public BoltDeclarer addConfigurations(Map<String, Object> conf) {
+            _component.componentConfs.add(conf);
+            return this;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java b/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java
new file mode 100644
index 0000000..71919a0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.coordination;
+
+import org.apache.storm.topology.FailedException;
+import java.util.Map.Entry;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.generated.GlobalStreamId;
+import java.util.Collection;
+import org.apache.storm.Constants;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.task.IOutputCollector;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.TimeCacheMap;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Coordination requires the request ids to be globally unique for awhile. This is so it doesn't get confused
+ * in the case of retries.
+ */
+public class CoordinatedBolt implements IRichBolt {
+    public static final Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class);
+
+    public static interface FinishedCallback {
+        void finishedId(Object id);
+    }
+
+    public static interface TimeoutCallback {
+        void timeoutId(Object id);
+    }
+    
+    
+    public static class SourceArgs implements Serializable {
+        public boolean singleCount;
+
+        protected SourceArgs(boolean singleCount) {
+            this.singleCount = singleCount;
+        }
+
+        public static SourceArgs single() {
+            return new SourceArgs(true);
+        }
+
+        public static SourceArgs all() {
+            return new SourceArgs(false);
+        }
+        
+        @Override
+        public String toString() {
+            return "<Single: " + singleCount + ">";
+        }
+    }
+
+    public class CoordinatedOutputCollector implements IOutputCollector {
+        IOutputCollector _delegate;
+
+        public CoordinatedOutputCollector(IOutputCollector delegate) {
+            _delegate = delegate;
+        }
+
+        public List<Integer> emit(String stream, Collection<Tuple> anchors, List<Object> tuple) {
+            List<Integer> tasks = _delegate.emit(stream, anchors, tuple);
+            updateTaskCounts(tuple.get(0), tasks);
+            return tasks;
+        }
+
+        public void emitDirect(int task, String stream, Collection<Tuple> anchors, List<Object> tuple) {
+            updateTaskCounts(tuple.get(0), Arrays.asList(task));
+            _delegate.emitDirect(task, stream, anchors, tuple);
+        }
+
+        public void ack(Tuple tuple) {
+            Object id = tuple.getValue(0);
+            synchronized(_tracked) {
+                TrackingInfo track = _tracked.get(id);
+                if (track != null)
+                    track.receivedTuples++;
+            }
+            boolean failed = checkFinishId(tuple, TupleType.REGULAR);
+            if(failed) {
+                _delegate.fail(tuple);                
+            } else {
+                _delegate.ack(tuple);
+            }
+        }
+
+        public void fail(Tuple tuple) {
+            Object id = tuple.getValue(0);
+            synchronized(_tracked) {
+                TrackingInfo track = _tracked.get(id);
+                if (track != null)
+                    track.failed = true;
+            }
+            checkFinishId(tuple, TupleType.REGULAR);
+            _delegate.fail(tuple);
+        }
+
+        public void resetTimeout(Tuple tuple) {
+            _delegate.resetTimeout(tuple);
+        }
+        
+        public void reportError(Throwable error) {
+            _delegate.reportError(error);
+        }
+
+
+        private void updateTaskCounts(Object id, List<Integer> tasks) {
+            synchronized(_tracked) {
+                TrackingInfo track = _tracked.get(id);
+                if (track != null) {
+                    Map<Integer, Integer> taskEmittedTuples = track.taskEmittedTuples;
+                    for(Integer task: tasks) {
+                        int newCount = Utils.get(taskEmittedTuples, task, 0) + 1;
+                        taskEmittedTuples.put(task, newCount);
+                    }
+                }
+            }
+        }
+    }
+
+    private Map<String, SourceArgs> _sourceArgs;
+    private IdStreamSpec _idStreamSpec;
+    private IRichBolt _delegate;
+    private Integer _numSourceReports;
+    private List<Integer> _countOutTasks = new ArrayList<>();
+    private OutputCollector _collector;
+    private TimeCacheMap<Object, TrackingInfo> _tracked;
+
+    public static class TrackingInfo {
+        int reportCount = 0;
+        int expectedTupleCount = 0;
+        int receivedTuples = 0;
+        boolean failed = false;
+        Map<Integer, Integer> taskEmittedTuples = new HashMap<>();
+        boolean receivedId = false;
+        boolean finished = false;
+        List<Tuple> ackTuples = new ArrayList<>();
+        
+        @Override
+        public String toString() {
+            return "reportCount: " + reportCount + "\n" +
+                   "expectedTupleCount: " + expectedTupleCount + "\n" +
+                   "receivedTuples: " + receivedTuples + "\n" +
+                   "failed: " + failed + "\n" +
+                   taskEmittedTuples.toString();
+        }
+    }
+
+    
+    public static class IdStreamSpec implements Serializable {
+        GlobalStreamId _id;
+        
+        public GlobalStreamId getGlobalStreamId() {
+            return _id;
+        }
+
+        public static IdStreamSpec makeDetectSpec(String component, String stream) {
+            return new IdStreamSpec(component, stream);
+        }        
+        
+        protected IdStreamSpec(String component, String stream) {
+            _id = new GlobalStreamId(component, stream);
+        }
+    }
+    
+    public CoordinatedBolt(IRichBolt delegate) {
+        this(delegate, null, null);
+    }
+
+    public CoordinatedBolt(IRichBolt delegate, String sourceComponent, SourceArgs sourceArgs, IdStreamSpec idStreamSpec) {
+        this(delegate, singleSourceArgs(sourceComponent, sourceArgs), idStreamSpec);
+    }
+    
+    public CoordinatedBolt(IRichBolt delegate, Map<String, SourceArgs> sourceArgs, IdStreamSpec idStreamSpec) {
+        _sourceArgs = sourceArgs;
+        if(_sourceArgs==null) _sourceArgs = new HashMap<>();
+        _delegate = delegate;
+        _idStreamSpec = idStreamSpec;
+    }
+    
+    public void prepare(Map config, TopologyContext context, OutputCollector collector) {
+        TimeCacheMap.ExpiredCallback<Object, TrackingInfo> callback = null;
+        if(_delegate instanceof TimeoutCallback) {
+            callback = new TimeoutItems();
+        }
+        _tracked = new TimeCacheMap<>(context.maxTopologyMessageTimeout(), callback);
+        _collector = collector;
+        _delegate.prepare(config, context, new OutputCollector(new CoordinatedOutputCollector(collector)));
+        for(String component: Utils.get(context.getThisTargets(),
+                                        Constants.COORDINATED_STREAM_ID,
+                                        new HashMap<String, Grouping>())
+                                        .keySet()) {
+            for(Integer task: context.getComponentTasks(component)) {
+                _countOutTasks.add(task);
+            }
+        }
+        if(!_sourceArgs.isEmpty()) {
+            _numSourceReports = 0;
+            for(Entry<String, SourceArgs> entry: _sourceArgs.entrySet()) {
+                if(entry.getValue().singleCount) {
+                    _numSourceReports+=1;
+                } else {
+                    _numSourceReports+=context.getComponentTasks(entry.getKey()).size();
+                }
+            }
+        }
+    }
+
+    private boolean checkFinishId(Tuple tup, TupleType type) {
+        Object id = tup.getValue(0);
+        boolean failed = false;
+        
+        synchronized(_tracked) {
+            TrackingInfo track = _tracked.get(id);
+            try {
+                if(track!=null) {
+                    boolean delayed = false;
+                    if(_idStreamSpec==null && type == TupleType.COORD || _idStreamSpec!=null && type==TupleType.ID) {
+                        track.ackTuples.add(tup);
+                        delayed = true;
+                    }
+                    if(track.failed) {
+                        failed = true;
+                        for(Tuple t: track.ackTuples) {
+                            _collector.fail(t);
+                        }
+                        _tracked.remove(id);
+                    } else if(track.receivedId
+                             && (_sourceArgs.isEmpty() ||
+                                  track.reportCount==_numSourceReports &&
+                                  track.expectedTupleCount == track.receivedTuples)){
+                        if(_delegate instanceof FinishedCallback) {
+                            ((FinishedCallback)_delegate).finishedId(id);
+                        }
+                        if(!(_sourceArgs.isEmpty() || type!=TupleType.REGULAR)) {
+                            throw new IllegalStateException("Coordination condition met on a non-coordinating tuple. Should be impossible");
+                        }
+                        Iterator<Integer> outTasks = _countOutTasks.iterator();
+                        while(outTasks.hasNext()) {
+                            int task = outTasks.next();
+                            int numTuples = Utils.get(track.taskEmittedTuples, task, 0);
+                            _collector.emitDirect(task, Constants.COORDINATED_STREAM_ID, tup, new Values(id, numTuples));
+                        }
+                        for(Tuple t: track.ackTuples) {
+                            _collector.ack(t);
+                        }
+                        track.finished = true;
+                        _tracked.remove(id);
+                    }
+                    if(!delayed && type!=TupleType.REGULAR) {
+                        if(track.failed) {
+                            _collector.fail(tup);
+                        } else {
+                            _collector.ack(tup);                            
+                        }
+                    }
+                } else {
+                    if(type!=TupleType.REGULAR) _collector.fail(tup);
+                }
+            } catch(FailedException e) {
+                LOG.error("Failed to finish batch", e);
+                for(Tuple t: track.ackTuples) {
+                    _collector.fail(t);
+                }
+                _tracked.remove(id);
+                failed = true;
+            }
+        }
+        return failed;
+    }
+
+    public void execute(Tuple tuple) {
+        Object id = tuple.getValue(0);
+        TrackingInfo track;
+        TupleType type = getTupleType(tuple);
+        synchronized(_tracked) {
+            track = _tracked.get(id);
+            if(track==null) {
+                track = new TrackingInfo();
+                if(_idStreamSpec==null) track.receivedId = true;
+                _tracked.put(id, track);
+            }
+        }
+        
+        if(type==TupleType.ID) {
+            synchronized(_tracked) {
+                track.receivedId = true;
+            }
+            checkFinishId(tuple, type);            
+        } else if(type==TupleType.COORD) {
+            int count = (Integer) tuple.getValue(1);
+            synchronized(_tracked) {
+                track.reportCount++;
+                track.expectedTupleCount+=count;
+            }
+            checkFinishId(tuple, type);
+        } else {            
+            synchronized(_tracked) {
+                _delegate.execute(tuple);
+            }
+        }
+    }
+
+    public void cleanup() {
+        _delegate.cleanup();
+        _tracked.cleanup();
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        _delegate.declareOutputFields(declarer);
+        declarer.declareStream(Constants.COORDINATED_STREAM_ID, true, new Fields("id", "count"));
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return _delegate.getComponentConfiguration();
+    }
+    
+    private static Map<String, SourceArgs> singleSourceArgs(String sourceComponent, SourceArgs sourceArgs) {
+        Map<String, SourceArgs> ret = new HashMap<>();
+        ret.put(sourceComponent, sourceArgs);
+        return ret;
+    }
+    
+    private class TimeoutItems implements TimeCacheMap.ExpiredCallback<Object, TrackingInfo> {
+        @Override
+        public void expire(Object id, TrackingInfo val) {
+            synchronized(_tracked) {
+                // the combination of the lock and the finished flag ensure that
+                // an id is never timed out if it has been finished
+                val.failed = true;
+                if(!val.finished) {
+                    ((TimeoutCallback) _delegate).timeoutId(id);
+                }
+            }
+        }
+    }
+    
+    private TupleType getTupleType(Tuple tuple) {
+        if(_idStreamSpec!=null
+                && tuple.getSourceGlobalStreamId().equals(_idStreamSpec._id)) {
+            return TupleType.ID;
+        } else if(!_sourceArgs.isEmpty()
+                && tuple.getSourceStreamId().equals(Constants.COORDINATED_STREAM_ID)) {
+            return TupleType.COORD;
+        } else {
+            return TupleType.REGULAR;
+        }
+    }
+    
+    static enum TupleType {
+        REGULAR,
+        ID,
+        COORD
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/coordination/IBatchBolt.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/IBatchBolt.java b/storm-client/src/jvm/org/apache/storm/coordination/IBatchBolt.java
new file mode 100644
index 0000000..7ea46b0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/coordination/IBatchBolt.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.coordination;
+
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.IComponent;
+import org.apache.storm.tuple.Tuple;
+import java.io.Serializable;
+import java.util.Map;
+
+public interface IBatchBolt<T> extends Serializable, IComponent {
+    void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, T id);
+    void execute(Tuple tuple);
+    void finishBatch();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/Acker.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/Acker.java b/storm-client/src/jvm/org/apache/storm/daemon/Acker.java
new file mode 100644
index 0000000..339e91c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/Acker.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon;
+
+import org.apache.storm.task.IBolt;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.RotatingMap;
+import org.apache.storm.utils.TupleUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class Acker implements IBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(Acker.class);
+
+    private static final long serialVersionUID = 4430906880683183091L;
+
+    public static final String ACKER_COMPONENT_ID = "__acker";
+    public static final String ACKER_INIT_STREAM_ID = "__ack_init";
+    public static final String ACKER_ACK_STREAM_ID = "__ack_ack";
+    public static final String ACKER_FAIL_STREAM_ID = "__ack_fail";
+    public static final String ACKER_RESET_TIMEOUT_STREAM_ID = "__ack_reset_timeout";
+
+    public static final int TIMEOUT_BUCKET_NUM = 3;
+
+    private OutputCollector collector;
+    private RotatingMap<Object, AckObject> pending;
+
+    private class AckObject {
+        public long val = 0L;
+        public Integer spoutTask = null;
+        public boolean failed = false;
+        public long startTime = System.currentTimeMillis();
+
+        // val xor value
+        public void updateAck(Long value) {
+            val = Utils.bitXor(val, value);
+        }
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.collector = collector;
+        this.pending = new RotatingMap<Object, AckObject>(TIMEOUT_BUCKET_NUM);
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        if (TupleUtils.isTick(input)) {
+            Map<Object, AckObject> tmp = pending.rotate();
+            LOG.debug("Number of timeout tuples:{}", tmp.size());
+            return;
+        }
+
+        String streamId = input.getSourceStreamId();
+        Object id = input.getValue(0);
+        AckObject curr = pending.get(id);
+        if (ACKER_INIT_STREAM_ID.equals(streamId)) {
+            if (curr == null) {
+                curr = new AckObject();
+                curr.val = input.getLong(1);
+                curr.spoutTask = input.getInteger(2);
+                pending.put(id, curr);
+            } else {
+                // If receiving bolt's ack before the init message from spout, just update the xor value.
+                curr.updateAck(input.getLong(1));
+                curr.spoutTask = input.getInteger(2);
+            }
+        } else if (ACKER_ACK_STREAM_ID.equals(streamId)) {
+            if (curr != null) {
+                curr.updateAck(input.getLong(1));
+            } else {
+                curr = new AckObject();
+                curr.val = input.getLong(1);
+                pending.put(id, curr);
+            }
+        } else if (ACKER_FAIL_STREAM_ID.equals(streamId)) {
+            // For the case that ack_fail message arrives before ack_init
+            if (curr == null) {
+                curr = new AckObject();
+            }
+            curr.failed = true;
+            pending.put(id, curr);
+        } else if (ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
+            if (curr == null) {
+                curr = new AckObject();
+            }
+            pending.put(id, curr);
+        } else {
+            LOG.warn("Unknown source stream {} from task-{}", streamId, input.getSourceTask());
+            return;
+        }
+
+        Integer task = curr.spoutTask;
+        if (curr != null && task != null) {
+            Values tuple = new Values(id, getTimeDeltaMillis(curr.startTime));
+            if (curr.val == 0) {
+                pending.remove(id);
+                collector.emitDirect(task, ACKER_ACK_STREAM_ID, tuple);
+            } else if (curr.failed) {
+                pending.remove(id);
+                collector.emitDirect(task, ACKER_FAIL_STREAM_ID, tuple);
+            } else if(ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
+                collector.emitDirect(task, ACKER_RESET_TIMEOUT_STREAM_ID, tuple);
+            }
+        }
+
+        collector.ack(input);
+    }
+
+    @Override
+    public void cleanup() {
+        LOG.info("Acker: cleanup successfully");
+    }
+
+    private long getTimeDeltaMillis(long startTimeMillis) {
+        return System.currentTimeMillis() - startTimeMillis;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/DaemonCommon.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/DaemonCommon.java b/storm-client/src/jvm/org/apache/storm/daemon/DaemonCommon.java
new file mode 100644
index 0000000..d1b71a7
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/DaemonCommon.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon;
+
+public interface DaemonCommon {
+    public boolean isWaiting();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java b/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
new file mode 100644
index 0000000..c4fdaf9
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.Sets;
+
+import org.apache.storm.Config;
+import org.apache.storm.Thrift;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.grouping.CustomStreamGrouping;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.grouping.LoadAwareShuffleGrouping;
+import org.apache.storm.grouping.LoadMapping;
+import org.apache.storm.grouping.ShuffleGrouping;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.TupleUtils;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+public class GrouperFactory {
+
+    public static LoadAwareCustomStreamGrouping mkGrouper(WorkerTopologyContext context, String componentId, String streamId, Fields outFields,
+                                    Grouping thriftGrouping,
+                                    List<Integer> unsortedTargetTasks,
+                                    Map topoConf) {
+        List<Integer> targetTasks = Ordering.natural().sortedCopy(unsortedTargetTasks);
+        final boolean isNotLoadAware = (null != topoConf.get(Config.TOPOLOGY_DISABLE_LOADAWARE_MESSAGING) && (boolean) topoConf
+            .get(Config.TOPOLOGY_DISABLE_LOADAWARE_MESSAGING));
+        CustomStreamGrouping result = null;
+        switch (Thrift.groupingType(thriftGrouping)) {
+            case FIELDS:
+                if (Thrift.isGlobalGrouping(thriftGrouping)) {
+                    result = new GlobalGrouper();
+                } else {
+                    result = new FieldsGrouper(outFields, thriftGrouping);
+                }
+                break;
+            case SHUFFLE:
+                if (isNotLoadAware) {
+                    result = new ShuffleGrouping();
+                } else {
+                    result = new LoadAwareShuffleGrouping();
+                }
+                break;
+            case ALL:
+                result = new AllGrouper();
+                break;
+            case LOCAL_OR_SHUFFLE:
+                // Prefer local tasks as target tasks if possible
+                Set<Integer> sameTasks = Sets.intersection(Sets.newHashSet(targetTasks), Sets.newHashSet(context.getThisWorkerTasks()));
+                targetTasks = (sameTasks.isEmpty()) ? targetTasks : new ArrayList<>(sameTasks);
+                if (isNotLoadAware) {
+                    result = new ShuffleGrouping();
+                } else {
+                    result = new LoadAwareShuffleGrouping();
+                }
+                break;
+            case NONE:
+                result = new NoneGrouper();
+                break;
+            case CUSTOM_OBJECT:
+                result = (CustomStreamGrouping) Thrift.instantiateJavaObject(thriftGrouping.get_custom_object());
+                break;
+            case CUSTOM_SERIALIZED:
+                result = Utils.javaDeserialize(thriftGrouping.get_custom_serialized(), CustomStreamGrouping.class);
+                break;
+            case DIRECT:
+                result = DIRECT;
+                break;
+            default:
+                result = null;
+                break;
+        }
+
+        if (null != result) {
+            result.prepare(context, new GlobalStreamId(componentId, streamId), targetTasks);
+        }
+
+        if (result instanceof LoadAwareCustomStreamGrouping) {
+            return (LoadAwareCustomStreamGrouping) result;
+        } else {
+            return new BasicLoadAwareCustomStreamGrouping (result);
+        }
+    }
+
+    /**
+     * A bridge between CustomStreamGrouping and LoadAwareCustomStreamGrouping
+     */
+    public static class BasicLoadAwareCustomStreamGrouping implements LoadAwareCustomStreamGrouping {
+
+        private final CustomStreamGrouping customStreamGrouping;
+
+        public BasicLoadAwareCustomStreamGrouping(CustomStreamGrouping customStreamGrouping) {
+            this.customStreamGrouping = customStreamGrouping;
+        }
+
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values, LoadMapping load) {
+            return customStreamGrouping.chooseTasks(taskId, values);
+        }
+
+        @Override
+        public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+            customStreamGrouping.prepare(context, stream, targetTasks);
+        }
+
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values) {
+            return customStreamGrouping.chooseTasks(taskId, values);
+        }
+    }
+
+    public static class FieldsGrouper implements CustomStreamGrouping {
+
+        private Fields outFields;
+        private List<Integer> targetTasks;
+        private Fields groupFields;
+        private int numTasks;
+
+        public FieldsGrouper(Fields outFields, Grouping thriftGrouping) {
+            this.outFields = outFields;
+            this.groupFields = new Fields(Thrift.fieldGrouping(thriftGrouping));
+
+        }
+
+        @Override
+        public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+            this.targetTasks = targetTasks;
+            this.numTasks = targetTasks.size();
+        }
+
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values) {
+            int targetTaskIndex = TupleUtils.chooseTaskIndex(outFields.select(groupFields, values), numTasks);
+            return Collections.singletonList(targetTasks.get(targetTaskIndex));
+        }
+
+    }
+
+    public static class GlobalGrouper implements CustomStreamGrouping {
+
+        private List<Integer> targetTasks;
+
+        public GlobalGrouper() {
+        }
+
+        @Override
+        public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+            this.targetTasks = targetTasks;
+        }
+
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values) {
+            if (targetTasks.isEmpty()) {
+                return null;
+            }
+            // It's possible for target to have multiple tasks if it reads multiple sources
+            return Collections.singletonList(targetTasks.get(0));
+        }
+    }
+
+    public static class NoneGrouper implements CustomStreamGrouping {
+
+        private List<Integer> targetTasks;
+        private int numTasks;
+        private final Random random;
+
+        public NoneGrouper() {
+            random = new Random();
+        }
+
+        @Override
+        public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+            this.targetTasks = targetTasks;
+            this.numTasks = targetTasks.size();
+        }
+
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values) {
+            int index = random.nextInt(numTasks);
+            return Collections.singletonList(targetTasks.get(index));
+        }
+    }
+
+    public static class AllGrouper implements CustomStreamGrouping {
+
+        private List<Integer> targetTasks;
+
+        @Override
+        public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+            this.targetTasks = targetTasks;
+        }
+
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values) {
+            return targetTasks;
+        }
+    }
+
+    // A no-op grouper
+    public static final LoadAwareCustomStreamGrouping DIRECT = new LoadAwareCustomStreamGrouping() {
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values, LoadMapping load) {
+            return null;
+        }
+
+        @Override
+        public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+
+        }
+
+        @Override
+        public List<Integer> chooseTasks(int taskId, List<Object> values) {
+            return null;
+        }
+
+    };
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/Shutdownable.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/Shutdownable.java b/storm-client/src/jvm/org/apache/storm/daemon/Shutdownable.java
new file mode 100644
index 0000000..fe9e6c9
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/Shutdownable.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon;
+
+public interface Shutdownable {
+    public void shutdown();
+}


[30/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/DistributedRPC.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/DistributedRPC.java b/storm-client/src/jvm/org/apache/storm/generated/DistributedRPC.java
new file mode 100644
index 0000000..6ef2559
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/DistributedRPC.java
@@ -0,0 +1,1328 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class DistributedRPC {
+
+  public interface Iface {
+
+    public String execute(String functionName, String funcArgs) throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void execute(String functionName, String funcArgs, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    public String execute(String functionName, String funcArgs) throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_execute(functionName, funcArgs);
+      return recv_execute();
+    }
+
+    public void send_execute(String functionName, String funcArgs) throws org.apache.thrift.TException
+    {
+      execute_args args = new execute_args();
+      args.set_functionName(functionName);
+      args.set_funcArgs(funcArgs);
+      sendBase("execute", args);
+    }
+
+    public String recv_execute() throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException
+    {
+      execute_result result = new execute_result();
+      receiveBase(result, "execute");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "execute failed: unknown result");
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void execute(String functionName, String funcArgs, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      execute_call method_call = new execute_call(functionName, funcArgs, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class execute_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String functionName;
+      private String funcArgs;
+      public execute_call(String functionName, String funcArgs, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.functionName = functionName;
+        this.funcArgs = funcArgs;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("execute", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        execute_args args = new execute_args();
+        args.set_functionName(functionName);
+        args.set_funcArgs(funcArgs);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public String getResult() throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_execute();
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("execute", new execute());
+      return processMap;
+    }
+
+    public static class execute<I extends Iface> extends org.apache.thrift.ProcessFunction<I, execute_args> {
+      public execute() {
+        super("execute");
+      }
+
+      public execute_args getEmptyArgsInstance() {
+        return new execute_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public execute_result getResult(I iface, execute_args args) throws org.apache.thrift.TException {
+        execute_result result = new execute_result();
+        try {
+          result.success = iface.execute(args.functionName, args.funcArgs);
+        } catch (DRPCExecutionException e) {
+          result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
+  }
+
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName());
+    public AsyncProcessor(I iface) {
+      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+    }
+
+    protected AsyncProcessor(I iface, Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends AsyncIface> Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      processMap.put("execute", new execute());
+      return processMap;
+    }
+
+    public static class execute<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, execute_args, String> {
+      public execute() {
+        super("execute");
+      }
+
+      public execute_args getEmptyArgsInstance() {
+        return new execute_args();
+      }
+
+      public AsyncMethodCallback<String> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<String>() { 
+          public void onComplete(String o) {
+            execute_result result = new execute_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            execute_result result = new execute_result();
+            if (e instanceof DRPCExecutionException) {
+                        result.e = (DRPCExecutionException) e;
+                        result.set_e_isSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, execute_args args, org.apache.thrift.async.AsyncMethodCallback<String> resultHandler) throws TException {
+        iface.execute(args.functionName, args.funcArgs,resultHandler);
+      }
+    }
+
+  }
+
+  public static class execute_args implements org.apache.thrift.TBase<execute_args, execute_args._Fields>, java.io.Serializable, Cloneable, Comparable<execute_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("execute_args");
+
+    private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift.protocol.TField("funcArgs", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new execute_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new execute_argsTupleSchemeFactory());
+    }
+
+    private String functionName; // required
+    private String funcArgs; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      FUNCTION_NAME((short)1, "functionName"),
+      FUNC_ARGS((short)2, "funcArgs");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // FUNCTION_NAME
+            return FUNCTION_NAME;
+          case 2: // FUNC_ARGS
+            return FUNC_ARGS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift.meta_data.FieldMetaData("funcArgs", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(execute_args.class, metaDataMap);
+    }
+
+    public execute_args() {
+    }
+
+    public execute_args(
+      String functionName,
+      String funcArgs)
+    {
+      this();
+      this.functionName = functionName;
+      this.funcArgs = funcArgs;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public execute_args(execute_args other) {
+      if (other.is_set_functionName()) {
+        this.functionName = other.functionName;
+      }
+      if (other.is_set_funcArgs()) {
+        this.funcArgs = other.funcArgs;
+      }
+    }
+
+    public execute_args deepCopy() {
+      return new execute_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.functionName = null;
+      this.funcArgs = null;
+    }
+
+    public String get_functionName() {
+      return this.functionName;
+    }
+
+    public void set_functionName(String functionName) {
+      this.functionName = functionName;
+    }
+
+    public void unset_functionName() {
+      this.functionName = null;
+    }
+
+    /** Returns true if field functionName is set (has been assigned a value) and false otherwise */
+    public boolean is_set_functionName() {
+      return this.functionName != null;
+    }
+
+    public void set_functionName_isSet(boolean value) {
+      if (!value) {
+        this.functionName = null;
+      }
+    }
+
+    public String get_funcArgs() {
+      return this.funcArgs;
+    }
+
+    public void set_funcArgs(String funcArgs) {
+      this.funcArgs = funcArgs;
+    }
+
+    public void unset_funcArgs() {
+      this.funcArgs = null;
+    }
+
+    /** Returns true if field funcArgs is set (has been assigned a value) and false otherwise */
+    public boolean is_set_funcArgs() {
+      return this.funcArgs != null;
+    }
+
+    public void set_funcArgs_isSet(boolean value) {
+      if (!value) {
+        this.funcArgs = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case FUNCTION_NAME:
+        if (value == null) {
+          unset_functionName();
+        } else {
+          set_functionName((String)value);
+        }
+        break;
+
+      case FUNC_ARGS:
+        if (value == null) {
+          unset_funcArgs();
+        } else {
+          set_funcArgs((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case FUNCTION_NAME:
+        return get_functionName();
+
+      case FUNC_ARGS:
+        return get_funcArgs();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case FUNCTION_NAME:
+        return is_set_functionName();
+      case FUNC_ARGS:
+        return is_set_funcArgs();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof execute_args)
+        return this.equals((execute_args)that);
+      return false;
+    }
+
+    public boolean equals(execute_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_functionName = true && this.is_set_functionName();
+      boolean that_present_functionName = true && that.is_set_functionName();
+      if (this_present_functionName || that_present_functionName) {
+        if (!(this_present_functionName && that_present_functionName))
+          return false;
+        if (!this.functionName.equals(that.functionName))
+          return false;
+      }
+
+      boolean this_present_funcArgs = true && this.is_set_funcArgs();
+      boolean that_present_funcArgs = true && that.is_set_funcArgs();
+      if (this_present_funcArgs || that_present_funcArgs) {
+        if (!(this_present_funcArgs && that_present_funcArgs))
+          return false;
+        if (!this.funcArgs.equals(that.funcArgs))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_functionName = true && (is_set_functionName());
+      list.add(present_functionName);
+      if (present_functionName)
+        list.add(functionName);
+
+      boolean present_funcArgs = true && (is_set_funcArgs());
+      list.add(present_funcArgs);
+      if (present_funcArgs)
+        list.add(funcArgs);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(execute_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(other.is_set_functionName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_functionName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, other.functionName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_funcArgs()).compareTo(other.is_set_funcArgs());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_funcArgs()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.funcArgs, other.funcArgs);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("execute_args(");
+      boolean first = true;
+
+      sb.append("functionName:");
+      if (this.functionName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.functionName);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("funcArgs:");
+      if (this.funcArgs == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.funcArgs);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class execute_argsStandardSchemeFactory implements SchemeFactory {
+      public execute_argsStandardScheme getScheme() {
+        return new execute_argsStandardScheme();
+      }
+    }
+
+    private static class execute_argsStandardScheme extends StandardScheme<execute_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, execute_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // FUNCTION_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.functionName = iprot.readString();
+                struct.set_functionName_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // FUNC_ARGS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.funcArgs = iprot.readString();
+                struct.set_funcArgs_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, execute_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.functionName != null) {
+          oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC);
+          oprot.writeString(struct.functionName);
+          oprot.writeFieldEnd();
+        }
+        if (struct.funcArgs != null) {
+          oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC);
+          oprot.writeString(struct.funcArgs);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class execute_argsTupleSchemeFactory implements SchemeFactory {
+      public execute_argsTupleScheme getScheme() {
+        return new execute_argsTupleScheme();
+      }
+    }
+
+    private static class execute_argsTupleScheme extends TupleScheme<execute_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, execute_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_functionName()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_funcArgs()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_functionName()) {
+          oprot.writeString(struct.functionName);
+        }
+        if (struct.is_set_funcArgs()) {
+          oprot.writeString(struct.funcArgs);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, execute_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.functionName = iprot.readString();
+          struct.set_functionName_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.funcArgs = iprot.readString();
+          struct.set_funcArgs_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class execute_result implements org.apache.thrift.TBase<execute_result, execute_result._Fields>, java.io.Serializable, Cloneable, Comparable<execute_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("execute_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
+    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new execute_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new execute_resultTupleSchemeFactory());
+    }
+
+    private String success; // required
+    private DRPCExecutionException e; // required
+    private AuthorizationException aze; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      E((short)1, "e"),
+      AZE((short)2, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // E
+            return E;
+          case 2: // AZE
+            return AZE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(execute_result.class, metaDataMap);
+    }
+
+    public execute_result() {
+    }
+
+    public execute_result(
+      String success,
+      DRPCExecutionException e,
+      AuthorizationException aze)
+    {
+      this();
+      this.success = success;
+      this.e = e;
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public execute_result(execute_result other) {
+      if (other.is_set_success()) {
+        this.success = other.success;
+      }
+      if (other.is_set_e()) {
+        this.e = new DRPCExecutionException(other.e);
+      }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public execute_result deepCopy() {
+      return new execute_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.e = null;
+      this.aze = null;
+    }
+
+    public String get_success() {
+      return this.success;
+    }
+
+    public void set_success(String success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public DRPCExecutionException get_e() {
+      return this.e;
+    }
+
+    public void set_e(DRPCExecutionException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been assigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public AuthorizationException get_aze() {
+      return this.aze;
+    }
+
+    public void set_aze(AuthorizationException aze) {
+      this.aze = aze;
+    }
+
+    public void unset_aze() {
+      this.aze = null;
+    }
+
+    /** Returns true if field aze is set (has been assigned a value) and false otherwise */
+    public boolean is_set_aze() {
+      return this.aze != null;
+    }
+
+    public void set_aze_isSet(boolean value) {
+      if (!value) {
+        this.aze = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((String)value);
+        }
+        break;
+
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((DRPCExecutionException)value);
+        }
+        break;
+
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case E:
+        return get_e();
+
+      case AZE:
+        return get_aze();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case E:
+        return is_set_e();
+      case AZE:
+        return is_set_aze();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof execute_result)
+        return this.equals((execute_result)that);
+      return false;
+    }
+
+    public boolean equals(execute_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      boolean this_present_aze = true && this.is_set_aze();
+      boolean that_present_aze = true && that.is_set_aze();
+      if (this_present_aze || that_present_aze) {
+        if (!(this_present_aze && that_present_aze))
+          return false;
+        if (!this.aze.equals(that.aze))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (is_set_success());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_e = true && (is_set_e());
+      list.add(present_e);
+      if (present_e)
+        list.add(e);
+
+      boolean present_aze = true && (is_set_aze());
+      list.add(present_aze);
+      if (present_aze)
+        list.add(aze);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(execute_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("execute_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("aze:");
+      if (this.aze == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.aze);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class execute_resultStandardSchemeFactory implements SchemeFactory {
+      public execute_resultStandardScheme getScheme() {
+        return new execute_resultStandardScheme();
+      }
+    }
+
+    private static class execute_resultStandardScheme extends StandardScheme<execute_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, execute_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.success = iprot.readString();
+                struct.set_success_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // E
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.e = new DRPCExecutionException();
+                struct.e.read(iprot);
+                struct.set_e_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // AZE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.aze = new AuthorizationException();
+                struct.aze.read(iprot);
+                struct.set_aze_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, execute_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeString(struct.success);
+          oprot.writeFieldEnd();
+        }
+        if (struct.e != null) {
+          oprot.writeFieldBegin(E_FIELD_DESC);
+          struct.e.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.aze != null) {
+          oprot.writeFieldBegin(AZE_FIELD_DESC);
+          struct.aze.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class execute_resultTupleSchemeFactory implements SchemeFactory {
+      public execute_resultTupleScheme getScheme() {
+        return new execute_resultTupleScheme();
+      }
+    }
+
+    private static class execute_resultTupleScheme extends TupleScheme<execute_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, execute_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_success()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_e()) {
+          optionals.set(1);
+        }
+        if (struct.is_set_aze()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.is_set_success()) {
+          oprot.writeString(struct.success);
+        }
+        if (struct.is_set_e()) {
+          struct.e.write(oprot);
+        }
+        if (struct.is_set_aze()) {
+          struct.aze.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, execute_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.success = iprot.readString();
+          struct.set_success_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.e = new DRPCExecutionException();
+          struct.e.read(iprot);
+          struct.set_e_isSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.aze = new AuthorizationException();
+          struct.aze.read(iprot);
+          struct.set_aze_isSet(true);
+        }
+      }
+    }
+
+  }
+
+}


[14/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/StormBase.java b/storm-client/src/jvm/org/apache/storm/generated/StormBase.java
new file mode 100644
index 0000000..34b2358
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/StormBase.java
@@ -0,0 +1,1382 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField COMPONENT_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("component_executors", org.apache.thrift.protocol.TType.MAP, (short)4);
+  private static final org.apache.thrift.protocol.TField LAUNCH_TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("launch_time_secs", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ACTION_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_action_options", org.apache.thrift.protocol.TType.STRUCT, (short)7);
+  private static final org.apache.thrift.protocol.TField PREV_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("prev_status", org.apache.thrift.protocol.TType.I32, (short)8);
+  private static final org.apache.thrift.protocol.TField COMPONENT_DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("component_debug", org.apache.thrift.protocol.TType.MAP, (short)9);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new StormBaseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new StormBaseTupleSchemeFactory());
+  }
+
+  private String name; // required
+  private TopologyStatus status; // required
+  private int num_workers; // required
+  private Map<String,Integer> component_executors; // optional
+  private int launch_time_secs; // optional
+  private String owner; // optional
+  private TopologyActionOptions topology_action_options; // optional
+  private TopologyStatus prev_status; // optional
+  private Map<String,DebugOptions> component_debug; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    /**
+     * 
+     * @see TopologyStatus
+     */
+    STATUS((short)2, "status"),
+    NUM_WORKERS((short)3, "num_workers"),
+    COMPONENT_EXECUTORS((short)4, "component_executors"),
+    LAUNCH_TIME_SECS((short)5, "launch_time_secs"),
+    OWNER((short)6, "owner"),
+    TOPOLOGY_ACTION_OPTIONS((short)7, "topology_action_options"),
+    /**
+     * 
+     * @see TopologyStatus
+     */
+    PREV_STATUS((short)8, "prev_status"),
+    COMPONENT_DEBUG((short)9, "component_debug");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // STATUS
+          return STATUS;
+        case 3: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 4: // COMPONENT_EXECUTORS
+          return COMPONENT_EXECUTORS;
+        case 5: // LAUNCH_TIME_SECS
+          return LAUNCH_TIME_SECS;
+        case 6: // OWNER
+          return OWNER;
+        case 7: // TOPOLOGY_ACTION_OPTIONS
+          return TOPOLOGY_ACTION_OPTIONS;
+        case 8: // PREV_STATUS
+          return PREV_STATUS;
+        case 9: // COMPONENT_DEBUG
+          return COMPONENT_DEBUG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_WORKERS_ISSET_ID = 0;
+  private static final int __LAUNCH_TIME_SECS_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.COMPONENT_EXECUTORS,_Fields.LAUNCH_TIME_SECS,_Fields.OWNER,_Fields.TOPOLOGY_ACTION_OPTIONS,_Fields.PREV_STATUS,_Fields.COMPONENT_DEBUG};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.COMPONENT_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("component_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
+    tmpMap.put(_Fields.LAUNCH_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("launch_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOPOLOGY_ACTION_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("topology_action_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyActionOptions.class)));
+    tmpMap.put(_Fields.PREV_STATUS, new org.apache.thrift.meta_data.FieldMetaData("prev_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
+    tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormBase.class, metaDataMap);
+  }
+
+  public StormBase() {
+  }
+
+  public StormBase(
+    String name,
+    TopologyStatus status,
+    int num_workers)
+  {
+    this();
+    this.name = name;
+    this.status = status;
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StormBase(StormBase other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    if (other.is_set_status()) {
+      this.status = other.status;
+    }
+    this.num_workers = other.num_workers;
+    if (other.is_set_component_executors()) {
+      Map<String,Integer> __this__component_executors = new HashMap<String,Integer>(other.component_executors);
+      this.component_executors = __this__component_executors;
+    }
+    this.launch_time_secs = other.launch_time_secs;
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
+    if (other.is_set_topology_action_options()) {
+      this.topology_action_options = new TopologyActionOptions(other.topology_action_options);
+    }
+    if (other.is_set_prev_status()) {
+      this.prev_status = other.prev_status;
+    }
+    if (other.is_set_component_debug()) {
+      Map<String,DebugOptions> __this__component_debug = new HashMap<String,DebugOptions>(other.component_debug.size());
+      for (Map.Entry<String, DebugOptions> other_element : other.component_debug.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        DebugOptions other_element_value = other_element.getValue();
+
+        String __this__component_debug_copy_key = other_element_key;
+
+        DebugOptions __this__component_debug_copy_value = new DebugOptions(other_element_value);
+
+        __this__component_debug.put(__this__component_debug_copy_key, __this__component_debug_copy_value);
+      }
+      this.component_debug = __this__component_debug;
+    }
+  }
+
+  public StormBase deepCopy() {
+    return new StormBase(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.status = null;
+    set_num_workers_isSet(false);
+    this.num_workers = 0;
+    this.component_executors = null;
+    set_launch_time_secs_isSet(false);
+    this.launch_time_secs = 0;
+    this.owner = null;
+    this.topology_action_options = null;
+    this.prev_status = null;
+    this.component_debug = null;
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public TopologyStatus get_status() {
+    return this.status;
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public void set_status(TopologyStatus status) {
+    this.status = status;
+  }
+
+  public void unset_status() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_status() {
+    return this.status != null;
+  }
+
+  public void set_status_isSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_component_executors_size() {
+    return (this.component_executors == null) ? 0 : this.component_executors.size();
+  }
+
+  public void put_to_component_executors(String key, int val) {
+    if (this.component_executors == null) {
+      this.component_executors = new HashMap<String,Integer>();
+    }
+    this.component_executors.put(key, val);
+  }
+
+  public Map<String,Integer> get_component_executors() {
+    return this.component_executors;
+  }
+
+  public void set_component_executors(Map<String,Integer> component_executors) {
+    this.component_executors = component_executors;
+  }
+
+  public void unset_component_executors() {
+    this.component_executors = null;
+  }
+
+  /** Returns true if field component_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_executors() {
+    return this.component_executors != null;
+  }
+
+  public void set_component_executors_isSet(boolean value) {
+    if (!value) {
+      this.component_executors = null;
+    }
+  }
+
+  public int get_launch_time_secs() {
+    return this.launch_time_secs;
+  }
+
+  public void set_launch_time_secs(int launch_time_secs) {
+    this.launch_time_secs = launch_time_secs;
+    set_launch_time_secs_isSet(true);
+  }
+
+  public void unset_launch_time_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LAUNCH_TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field launch_time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_launch_time_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __LAUNCH_TIME_SECS_ISSET_ID);
+  }
+
+  public void set_launch_time_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LAUNCH_TIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = null;
+  }
+
+  /** Returns true if field owner is set (has been assigned a value) and false otherwise */
+  public boolean is_set_owner() {
+    return this.owner != null;
+  }
+
+  public void set_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
+  public TopologyActionOptions get_topology_action_options() {
+    return this.topology_action_options;
+  }
+
+  public void set_topology_action_options(TopologyActionOptions topology_action_options) {
+    this.topology_action_options = topology_action_options;
+  }
+
+  public void unset_topology_action_options() {
+    this.topology_action_options = null;
+  }
+
+  /** Returns true if field topology_action_options is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_action_options() {
+    return this.topology_action_options != null;
+  }
+
+  public void set_topology_action_options_isSet(boolean value) {
+    if (!value) {
+      this.topology_action_options = null;
+    }
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public TopologyStatus get_prev_status() {
+    return this.prev_status;
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public void set_prev_status(TopologyStatus prev_status) {
+    this.prev_status = prev_status;
+  }
+
+  public void unset_prev_status() {
+    this.prev_status = null;
+  }
+
+  /** Returns true if field prev_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_prev_status() {
+    return this.prev_status != null;
+  }
+
+  public void set_prev_status_isSet(boolean value) {
+    if (!value) {
+      this.prev_status = null;
+    }
+  }
+
+  public int get_component_debug_size() {
+    return (this.component_debug == null) ? 0 : this.component_debug.size();
+  }
+
+  public void put_to_component_debug(String key, DebugOptions val) {
+    if (this.component_debug == null) {
+      this.component_debug = new HashMap<String,DebugOptions>();
+    }
+    this.component_debug.put(key, val);
+  }
+
+  public Map<String,DebugOptions> get_component_debug() {
+    return this.component_debug;
+  }
+
+  public void set_component_debug(Map<String,DebugOptions> component_debug) {
+    this.component_debug = component_debug;
+  }
+
+  public void unset_component_debug() {
+    this.component_debug = null;
+  }
+
+  /** Returns true if field component_debug is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_debug() {
+    return this.component_debug != null;
+  }
+
+  public void set_component_debug_isSet(boolean value) {
+    if (!value) {
+      this.component_debug = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unset_status();
+      } else {
+        set_status((TopologyStatus)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case COMPONENT_EXECUTORS:
+      if (value == null) {
+        unset_component_executors();
+      } else {
+        set_component_executors((Map<String,Integer>)value);
+      }
+      break;
+
+    case LAUNCH_TIME_SECS:
+      if (value == null) {
+        unset_launch_time_secs();
+      } else {
+        set_launch_time_secs((Integer)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
+    case TOPOLOGY_ACTION_OPTIONS:
+      if (value == null) {
+        unset_topology_action_options();
+      } else {
+        set_topology_action_options((TopologyActionOptions)value);
+      }
+      break;
+
+    case PREV_STATUS:
+      if (value == null) {
+        unset_prev_status();
+      } else {
+        set_prev_status((TopologyStatus)value);
+      }
+      break;
+
+    case COMPONENT_DEBUG:
+      if (value == null) {
+        unset_component_debug();
+      } else {
+        set_component_debug((Map<String,DebugOptions>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return get_name();
+
+    case STATUS:
+      return get_status();
+
+    case NUM_WORKERS:
+      return get_num_workers();
+
+    case COMPONENT_EXECUTORS:
+      return get_component_executors();
+
+    case LAUNCH_TIME_SECS:
+      return get_launch_time_secs();
+
+    case OWNER:
+      return get_owner();
+
+    case TOPOLOGY_ACTION_OPTIONS:
+      return get_topology_action_options();
+
+    case PREV_STATUS:
+      return get_prev_status();
+
+    case COMPONENT_DEBUG:
+      return get_component_debug();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return is_set_name();
+    case STATUS:
+      return is_set_status();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case COMPONENT_EXECUTORS:
+      return is_set_component_executors();
+    case LAUNCH_TIME_SECS:
+      return is_set_launch_time_secs();
+    case OWNER:
+      return is_set_owner();
+    case TOPOLOGY_ACTION_OPTIONS:
+      return is_set_topology_action_options();
+    case PREV_STATUS:
+      return is_set_prev_status();
+    case COMPONENT_DEBUG:
+      return is_set_component_debug();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StormBase)
+      return this.equals((StormBase)that);
+    return false;
+  }
+
+  public boolean equals(StormBase that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_status = true && this.is_set_status();
+    boolean that_present_status = true && that.is_set_status();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    boolean this_present_num_workers = true;
+    boolean that_present_num_workers = true;
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_component_executors = true && this.is_set_component_executors();
+    boolean that_present_component_executors = true && that.is_set_component_executors();
+    if (this_present_component_executors || that_present_component_executors) {
+      if (!(this_present_component_executors && that_present_component_executors))
+        return false;
+      if (!this.component_executors.equals(that.component_executors))
+        return false;
+    }
+
+    boolean this_present_launch_time_secs = true && this.is_set_launch_time_secs();
+    boolean that_present_launch_time_secs = true && that.is_set_launch_time_secs();
+    if (this_present_launch_time_secs || that_present_launch_time_secs) {
+      if (!(this_present_launch_time_secs && that_present_launch_time_secs))
+        return false;
+      if (this.launch_time_secs != that.launch_time_secs)
+        return false;
+    }
+
+    boolean this_present_owner = true && this.is_set_owner();
+    boolean that_present_owner = true && that.is_set_owner();
+    if (this_present_owner || that_present_owner) {
+      if (!(this_present_owner && that_present_owner))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
+    boolean this_present_topology_action_options = true && this.is_set_topology_action_options();
+    boolean that_present_topology_action_options = true && that.is_set_topology_action_options();
+    if (this_present_topology_action_options || that_present_topology_action_options) {
+      if (!(this_present_topology_action_options && that_present_topology_action_options))
+        return false;
+      if (!this.topology_action_options.equals(that.topology_action_options))
+        return false;
+    }
+
+    boolean this_present_prev_status = true && this.is_set_prev_status();
+    boolean that_present_prev_status = true && that.is_set_prev_status();
+    if (this_present_prev_status || that_present_prev_status) {
+      if (!(this_present_prev_status && that_present_prev_status))
+        return false;
+      if (!this.prev_status.equals(that.prev_status))
+        return false;
+    }
+
+    boolean this_present_component_debug = true && this.is_set_component_debug();
+    boolean that_present_component_debug = true && that.is_set_component_debug();
+    if (this_present_component_debug || that_present_component_debug) {
+      if (!(this_present_component_debug && that_present_component_debug))
+        return false;
+      if (!this.component_debug.equals(that.component_debug))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_status = true && (is_set_status());
+    list.add(present_status);
+    if (present_status)
+      list.add(status.getValue());
+
+    boolean present_num_workers = true;
+    list.add(present_num_workers);
+    if (present_num_workers)
+      list.add(num_workers);
+
+    boolean present_component_executors = true && (is_set_component_executors());
+    list.add(present_component_executors);
+    if (present_component_executors)
+      list.add(component_executors);
+
+    boolean present_launch_time_secs = true && (is_set_launch_time_secs());
+    list.add(present_launch_time_secs);
+    if (present_launch_time_secs)
+      list.add(launch_time_secs);
+
+    boolean present_owner = true && (is_set_owner());
+    list.add(present_owner);
+    if (present_owner)
+      list.add(owner);
+
+    boolean present_topology_action_options = true && (is_set_topology_action_options());
+    list.add(present_topology_action_options);
+    if (present_topology_action_options)
+      list.add(topology_action_options);
+
+    boolean present_prev_status = true && (is_set_prev_status());
+    list.add(present_prev_status);
+    if (present_prev_status)
+      list.add(prev_status.getValue());
+
+    boolean present_component_debug = true && (is_set_component_debug());
+    list.add(present_component_debug);
+    if (present_component_debug)
+      list.add(component_debug);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(StormBase other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_executors()).compareTo(other.is_set_component_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_executors, other.component_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_launch_time_secs()).compareTo(other.is_set_launch_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_launch_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.launch_time_secs, other.launch_time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_action_options()).compareTo(other.is_set_topology_action_options());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_action_options()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_action_options, other.topology_action_options);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_prev_status()).compareTo(other.is_set_prev_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_prev_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.prev_status, other.prev_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_debug()).compareTo(other.is_set_component_debug());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_debug()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_debug, other.component_debug);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StormBase(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("status:");
+    if (this.status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.status);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_workers:");
+    sb.append(this.num_workers);
+    first = false;
+    if (is_set_component_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("component_executors:");
+      if (this.component_executors == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_executors);
+      }
+      first = false;
+    }
+    if (is_set_launch_time_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("launch_time_secs:");
+      sb.append(this.launch_time_secs);
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
+    if (is_set_topology_action_options()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_action_options:");
+      if (this.topology_action_options == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_action_options);
+      }
+      first = false;
+    }
+    if (is_set_prev_status()) {
+      if (!first) sb.append(", ");
+      sb.append("prev_status:");
+      if (this.prev_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.prev_status);
+      }
+      first = false;
+    }
+    if (is_set_component_debug()) {
+      if (!first) sb.append(", ");
+      sb.append("component_debug:");
+      if (this.component_debug == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_debug);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_status()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_workers()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class StormBaseStandardSchemeFactory implements SchemeFactory {
+    public StormBaseStandardScheme getScheme() {
+      return new StormBaseStandardScheme();
+    }
+  }
+
+  private static class StormBaseStandardScheme extends StandardScheme<StormBase> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, StormBase struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.status = org.apache.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+              struct.set_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NUM_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_workers = iprot.readI32();
+              struct.set_num_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // COMPONENT_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map674 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map674.size);
+                String _key675;
+                int _val676;
+                for (int _i677 = 0; _i677 < _map674.size; ++_i677)
+                {
+                  _key675 = iprot.readString();
+                  _val676 = iprot.readI32();
+                  struct.component_executors.put(_key675, _val676);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_component_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // LAUNCH_TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.launch_time_secs = iprot.readI32();
+              struct.set_launch_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // OWNER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.owner = iprot.readString();
+              struct.set_owner_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // TOPOLOGY_ACTION_OPTIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.topology_action_options = new TopologyActionOptions();
+              struct.topology_action_options.read(iprot);
+              struct.set_topology_action_options_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // PREV_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.prev_status = org.apache.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+              struct.set_prev_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // COMPONENT_DEBUG
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map678 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map678.size);
+                String _key679;
+                DebugOptions _val680;
+                for (int _i681 = 0; _i681 < _map678.size; ++_i681)
+                {
+                  _key679 = iprot.readString();
+                  _val680 = new DebugOptions();
+                  _val680.read(iprot);
+                  struct.component_debug.put(_key679, _val680);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_component_debug_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, StormBase struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.status != null) {
+        oprot.writeFieldBegin(STATUS_FIELD_DESC);
+        oprot.writeI32(struct.status.getValue());
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+      oprot.writeI32(struct.num_workers);
+      oprot.writeFieldEnd();
+      if (struct.component_executors != null) {
+        if (struct.is_set_component_executors()) {
+          oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.component_executors.size()));
+            for (Map.Entry<String, Integer> _iter682 : struct.component_executors.entrySet())
+            {
+              oprot.writeString(_iter682.getKey());
+              oprot.writeI32(_iter682.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_launch_time_secs()) {
+        oprot.writeFieldBegin(LAUNCH_TIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.launch_time_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.owner != null) {
+        if (struct.is_set_owner()) {
+          oprot.writeFieldBegin(OWNER_FIELD_DESC);
+          oprot.writeString(struct.owner);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_action_options != null) {
+        if (struct.is_set_topology_action_options()) {
+          oprot.writeFieldBegin(TOPOLOGY_ACTION_OPTIONS_FIELD_DESC);
+          struct.topology_action_options.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.prev_status != null) {
+        if (struct.is_set_prev_status()) {
+          oprot.writeFieldBegin(PREV_STATUS_FIELD_DESC);
+          oprot.writeI32(struct.prev_status.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.component_debug != null) {
+        if (struct.is_set_component_debug()) {
+          oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.component_debug.size()));
+            for (Map.Entry<String, DebugOptions> _iter683 : struct.component_debug.entrySet())
+            {
+              oprot.writeString(_iter683.getKey());
+              _iter683.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class StormBaseTupleSchemeFactory implements SchemeFactory {
+    public StormBaseTupleScheme getScheme() {
+      return new StormBaseTupleScheme();
+    }
+  }
+
+  private static class StormBaseTupleScheme extends TupleScheme<StormBase> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, StormBase struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.name);
+      oprot.writeI32(struct.status.getValue());
+      oprot.writeI32(struct.num_workers);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_component_executors()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_launch_time_secs()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_owner()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_topology_action_options()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_prev_status()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_component_debug()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
+      if (struct.is_set_component_executors()) {
+        {
+          oprot.writeI32(struct.component_executors.size());
+          for (Map.Entry<String, Integer> _iter684 : struct.component_executors.entrySet())
+          {
+            oprot.writeString(_iter684.getKey());
+            oprot.writeI32(_iter684.getValue());
+          }
+        }
+      }
+      if (struct.is_set_launch_time_secs()) {
+        oprot.writeI32(struct.launch_time_secs);
+      }
+      if (struct.is_set_owner()) {
+        oprot.writeString(struct.owner);
+      }
+      if (struct.is_set_topology_action_options()) {
+        struct.topology_action_options.write(oprot);
+      }
+      if (struct.is_set_prev_status()) {
+        oprot.writeI32(struct.prev_status.getValue());
+      }
+      if (struct.is_set_component_debug()) {
+        {
+          oprot.writeI32(struct.component_debug.size());
+          for (Map.Entry<String, DebugOptions> _iter685 : struct.component_debug.entrySet())
+          {
+            oprot.writeString(_iter685.getKey());
+            _iter685.getValue().write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, StormBase struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.name = iprot.readString();
+      struct.set_name_isSet(true);
+      struct.status = org.apache.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+      struct.set_status_isSet(true);
+      struct.num_workers = iprot.readI32();
+      struct.set_num_workers_isSet(true);
+      BitSet incoming = iprot.readBitSet(6);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map686 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+          struct.component_executors = new HashMap<String,Integer>(2*_map686.size);
+          String _key687;
+          int _val688;
+          for (int _i689 = 0; _i689 < _map686.size; ++_i689)
+          {
+            _key687 = iprot.readString();
+            _val688 = iprot.readI32();
+            struct.component_executors.put(_key687, _val688);
+          }
+        }
+        struct.set_component_executors_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.launch_time_secs = iprot.readI32();
+        struct.set_launch_time_secs_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.owner = iprot.readString();
+        struct.set_owner_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.topology_action_options = new TopologyActionOptions();
+        struct.topology_action_options.read(iprot);
+        struct.set_topology_action_options_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.prev_status = org.apache.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+        struct.set_prev_status_isSet(true);
+      }
+      if (incoming.get(5)) {
+        {
+          org.apache.thrift.protocol.TMap _map690 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.component_debug = new HashMap<String,DebugOptions>(2*_map690.size);
+          String _key691;
+          DebugOptions _val692;
+          for (int _i693 = 0; _i693 < _map690.size; ++_i693)
+          {
+            _key691 = iprot.readString();
+            _val692 = new DebugOptions();
+            _val692.read(iprot);
+            struct.component_debug.put(_key691, _val692);
+          }
+        }
+        struct.set_component_debug_isSet(true);
+      }
+    }
+  }
+
+}
+


[50/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index d65082f..f70d5e0 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -14,8 +14,10 @@ addons:
     - node1
 
 env:
+  - MODULES=storm-client
+  - MODULES=storm-server
   - MODULES=storm-core
-  - MODULES='!storm-core'
+  - MODULES='!storm-client,!storm-server,!storm-core'
   - MODULES='INTEGRATION-TEST'
 
 language: java

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/DEVELOPER.md
----------------------------------------------------------------------
diff --git a/DEVELOPER.md b/DEVELOPER.md
index 85022d8..9eb140d 100644
--- a/DEVELOPER.md
+++ b/DEVELOPER.md
@@ -272,7 +272,7 @@ If you wish to skip the unit tests you can do this by adding `-DskipTests` to th
 In case you modified `storm.thrift`, you have to regenerate thrift code as java and python code before compiling whole project.
 
 ```sh
-cd storm-core/src
+cd storm-client/src
 sh genthrift.sh
 ```
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/bin/storm.py
----------------------------------------------------------------------
diff --git a/bin/storm.py b/bin/storm.py
index a2d9243..41cd8f4 100755
--- a/bin/storm.py
+++ b/bin/storm.py
@@ -80,9 +80,10 @@ else:
 if (not os.path.isfile(os.path.join(USER_CONF_DIR, "storm.yaml"))):
     USER_CONF_DIR = CLUSTER_CONF_DIR
 
+STORM_WORKER_LIB_DIR = os.path.join(STORM_DIR, "lib-worker")
 STORM_LIB_DIR = os.path.join(STORM_DIR, "lib")
-STORM_DRPC_LIB_DIR = os.path.join(STORM_DIR, "lib-drpc-server")
 STORM_TOOLS_LIB_DIR = os.path.join(STORM_DIR, "lib-tools")
+STORM_WEBAPP_LIB_DIR = os.path.join(STORM_DIR, "lib-webapp")
 STORM_BIN_DIR = os.path.join(STORM_DIR, "bin")
 STORM_LOG4J2_CONF_DIR = os.path.join(STORM_DIR, "log4j2")
 STORM_SUPERVISOR_LOG_FILE = os.getenv('STORM_SUPERVISOR_LOG_FILE', "supervisor.log")
@@ -763,7 +764,7 @@ def drpc():
         "-DLog4jContextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector",
         "-Dlog4j.configurationFile=" + os.path.join(get_log4j2_conf_dir(), "cluster.xml")
     ]
-    allextrajars = get_jars_full(STORM_DRPC_LIB_DIR)
+    allextrajars = get_jars_full(STORM_WEBAPP_LIB_DIR)
     allextrajars.append(CLUSTER_CONF_DIR)
     exec_storm_class(
         "org.apache.storm.daemon.drpc.DRPCServer",

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Defining-a-non-jvm-language-dsl-for-storm.md
----------------------------------------------------------------------
diff --git a/docs/Defining-a-non-jvm-language-dsl-for-storm.md b/docs/Defining-a-non-jvm-language-dsl-for-storm.md
index 7096a43..311577b 100644
--- a/docs/Defining-a-non-jvm-language-dsl-for-storm.md
+++ b/docs/Defining-a-non-jvm-language-dsl-for-storm.md
@@ -3,7 +3,7 @@ title: Defining a Non-JVM DSL for Storm
 layout: documentation
 documentation: true
 ---
-The right place to start to learn how to make a non-JVM DSL for Storm is [storm-core/src/storm.thrift]({{page.git-blob-base}}/storm-core/src/storm.thrift). Since Storm topologies are just Thrift structures, and Nimbus is a Thrift daemon, you can create and submit topologies in any language.
+The right place to start to learn how to make a non-JVM DSL for Storm is [storm-client/src/storm.thrift]({{page.git-blob-base}}/storm-client/src/storm.thrift). Since Storm topologies are just Thrift structures, and Nimbus is a Thrift daemon, you can create and submit topologies in any language.
 
 When you create the Thrift structs for spouts and bolts, the code for the spout or bolt is specified in the ComponentObject struct:
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Lifecycle-of-a-topology.md
----------------------------------------------------------------------
diff --git a/docs/Lifecycle-of-a-topology.md b/docs/Lifecycle-of-a-topology.md
index dba1457..fe785f1 100644
--- a/docs/Lifecycle-of-a-topology.md
+++ b/docs/Lifecycle-of-a-topology.md
@@ -3,7 +3,7 @@ title: Lifecycle of a Storm Topology
 layout: documentation
 documentation: true
 ---
-(**NOTE**: this page is based on the 0.7.1 code; many things have changed since then, including a split between tasks and executors, and a reorganization of the code under `storm-core/src` rather than `src/`.)
+(**NOTE**: this page is based on the 0.7.1 code; many things have changed since then, including a split between tasks and executors, and a reorganization of the code under `storm-client/src` rather than `src/`.)
 
 This page explains in detail the lifecycle of a topology from running the "storm jar" command to uploading the topology to Nimbus to the supervisors starting/stopping workers to workers and tasks setting themselves up. It also explains how Nimbus monitors topologies and how topologies are shutdown when they are killed.
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Maven.md
----------------------------------------------------------------------
diff --git a/docs/Maven.md b/docs/Maven.md
index 0c09c2c..47fd1b1 100644
--- a/docs/Maven.md
+++ b/docs/Maven.md
@@ -9,7 +9,7 @@ To develop topologies, you'll need the Storm jars on your classpath. You should
 ```xml
 <dependency>
   <groupId>org.apache.storm</groupId>
-  <artifactId>storm-core</artifactId>
+  <artifactId>storm-client</artifactId>
   <version>{{page.version}}</version>
   <scope>provided</scope>
 </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Metrics.md
----------------------------------------------------------------------
diff --git a/docs/Metrics.md b/docs/Metrics.md
index 8537c7c..3142ef9 100644
--- a/docs/Metrics.md
+++ b/docs/Metrics.md
@@ -8,17 +8,17 @@ It's used internally to track the numbers you see in the Nimbus UI console: coun
 
 ### Metric Types
 
-Metrics have to implement [`IMetric`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero.
+Metrics have to implement [`IMetric`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero.
 
 Storm gives you these metric types:
 
-* [AssignableMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/AssignableMetric.java) -- set the metric to the explicit value you supply. Useful if it's an external value or in the case that you are already calculating the summary statistic yourself.
-* [CombinedMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/CombinedMetric.java) -- generic interface for metrics that can be updated associatively. 
-* [CountMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/CountMetric.java) -- a running total of the supplied values. Call `incr()` to increment by one, `incrBy(n)` to add/subtract the given number.
-  - [MultiCountMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java) -- a hashmap of count metrics.
-* [ReducedMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/ReducedMetric.java)
-  - [MeanReducer]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/MeanReducer.java) -- track a running average of values given to its `reduce()` method. (It accepts `Double`, `Integer` or `Long` values, and maintains the internal average as a `Double`.) Despite his reputation, the MeanReducer is actually a pretty nice guy in person.
-  - [MultiReducedMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java) -- a hashmap of reduced metrics.
+* [AssignableMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java) -- set the metric to the explicit value you supply. Useful if it's an external value or in the case that you are already calculating the summary statistic yourself.
+* [CombinedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java) -- generic interface for metrics that can be updated associatively. 
+* [CountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java) -- a running total of the supplied values. Call `incr()` to increment by one, `incrBy(n)` to add/subtract the given number.
+  - [MultiCountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java) -- a hashmap of count metrics.
+* [ReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java)
+  - [MeanReducer]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java) -- track a running average of values given to its `reduce()` method. (It accepts `Double`, `Integer` or `Long` values, and maintains the internal average as a `Double`.) Despite his reputation, the MeanReducer is actually a pretty nice guy in person.
+  - [MultiReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java) -- a hashmap of reduced metrics.
 
 
 ### Metrics Consumer
@@ -48,10 +48,10 @@ Storm appends MetricsConsumerBolt to your topology per each registered metrics c
 
 Storm provides some built-in metrics consumers for you to try out to see which metrics are provided in your topology.
 
-* [`LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- listens for all metrics and dumps them to log file with TSV (Tab Separated Values).
-* [`HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsConsumer.java) -- listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [`HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by HttpForwardingMetricsConsumer.
+* [`LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- listens for all metrics and dumps them to log file with TSV (Tab Separated Values).
+* [`HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/storm-client-misc/src/jvm/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java) -- listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [`HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by HttpForwardingMetricsConsumer.
 
-Also, Storm exposes the interface [`IMetricsConsumer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other great implementation of Metrics Consumers provided by Storm community. Some of examples are [versign/storm-graphite](https://github.com/verisign/storm-graphite), and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd).
+Also, Storm exposes the interface [`IMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other great implementation of Metrics Consumers provided by Storm community. Some of examples are [versign/storm-graphite](https://github.com/verisign/storm-graphite), and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd).
 
 When you implement your own metrics consumer, `argument` is passed to Object when [IMetricsConsumer#prepare](javadocs/org/apache/storm/metric/api/IMetricsConsumer.html#prepare-java.util.Map-java.lang.Object-org.apache.storm.task.TopologyContext-org.apache.storm.task.IErrorReporter-) is called, so you need to infer the Java type of configured value on yaml, and do explicit type casting.
 
@@ -121,7 +121,7 @@ C) Bucket size (seconds) for metrics is fixed to `Config.TOPOLOGY_BUILTIN_METRIC
 
 ### Builtin Metrics
 
-The [builtin metrics]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj) instrument Storm itself.
+The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) instrument Storm itself.
 
-[builtin_metrics.clj]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ack-spout-msg`]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/executor.clj#358)  in `clj/b/s/daemon/daemon/executor.clj`
+[BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Storm-Scheduler.md
----------------------------------------------------------------------
diff --git a/docs/Storm-Scheduler.md b/docs/Storm-Scheduler.md
index b0bd45e..7b8947e 100644
--- a/docs/Storm-Scheduler.md
+++ b/docs/Storm-Scheduler.md
@@ -4,10 +4,10 @@ layout: documentation
 documentation: true
 ---
 
-Storm now has 4 kinds of built-in schedulers: [DefaultScheduler]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/scheduler/DefaultScheduler.java), [IsolationScheduler]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/scheduler/IsolationScheduler.java), [MultitenantScheduler]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/scheduler/multitenant/MultitenantScheduler.java), [ResourceAwareScheduler](Resource_Aware_Scheduler_overview.html). 
+Storm now has 4 kinds of built-in schedulers: [DefaultScheduler]({{page.git-blob-base}}/storm-server/src/jvm/org/apache/storm/scheduler/DefaultScheduler.java), [IsolationScheduler]({{page.git-blob-base}}/storm-server/src/jvm/org/apache/storm/scheduler/IsolationScheduler.java), [MultitenantScheduler]({{page.git-blob-base}}/storm-server/src/jvm/org/apache/storm/scheduler/multitenant/MultitenantScheduler.java), [ResourceAwareScheduler](Resource_Aware_Scheduler_overview.html). 
 
 ## Pluggable scheduler
-You can implement your own scheduler to replace the default scheduler to assign executors to workers. You configure the class to use the "storm.scheduler" config in your storm.yaml, and your scheduler must implement the [IScheduler]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/scheduler/IScheduler.java) interface.
+You can implement your own scheduler to replace the default scheduler to assign executors to workers. You configure the class to use the "storm.scheduler" config in your storm.yaml, and your scheduler must implement the [IScheduler]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/scheduler/IScheduler.java) interface.
 
 ## Isolation Scheduler
 The isolation scheduler makes it easy and safe to share a cluster among many topologies. The isolation scheduler lets you specify which topologies should be "isolated", meaning that they run on a dedicated set of machines within the cluster where no other topologies will be running. These isolated topologies are given priority on the cluster, so resources will be allocated to isolated topologies if there's competition with non-isolated topologies, and resources will be taken away from non-isolated topologies if necessary to get resources for an isolated topology. Once all isolated topologies are allocated, the remaining machines on the cluster are shared among all non-isolated topologies.

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Stream-API.md
----------------------------------------------------------------------
diff --git a/docs/Stream-API.md b/docs/Stream-API.md
index f2e4cb6..2ecfb8c 100644
--- a/docs/Stream-API.md
+++ b/docs/Stream-API.md
@@ -109,7 +109,7 @@ Stream<Tuple3<String, Integer, Long>> stream = builder.newStream(new TestSpout()
 
 # <a name="streamapis"></a> Stream APIs 
 
-Storm's streaming apis (defined in [Stream](../storm-core/src/jvm/org/apache/storm/streams/Stream.java) and [PairStream](../storm-core/src/jvm/org/apache/storm/streams/PairStream.java)) currently support a wide range of operations such as transformations, filters, windowing, aggregations, branching, joins, stateful, output and debugging operations.
+Storm's streaming apis (defined in [Stream](../storm-client/src/jvm/org/apache/storm/streams/Stream.java) and [PairStream](../storm-client/src/jvm/org/apache/storm/streams/PairStream.java)) currently support a wide range of operations such as transformations, filters, windowing, aggregations, branching, joins, stateful, output and debugging operations.
 
 ## <a name="basictransformations"></a> Basic transformations 
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Structure-of-the-codebase.md
----------------------------------------------------------------------
diff --git a/docs/Structure-of-the-codebase.md b/docs/Structure-of-the-codebase.md
index 3040237..8c53002 100644
--- a/docs/Structure-of-the-codebase.md
+++ b/docs/Structure-of-the-codebase.md
@@ -15,18 +15,16 @@ The following sections explain each of these layers in more detail.
 
 ### storm.thrift
 
-The first place to look to understand the structure of Storm's codebase is the [storm.thrift]({{page.git-blob-base}}/storm-core/src/storm.thrift) file.
+The first place to look to understand the structure of Storm's codebase is the [storm.thrift]({{page.git-blob-base}}/storm-client/src/storm.thrift) file.
 
-Storm uses [this fork](https://github.com/nathanmarz/thrift/tree/storm) of Thrift (branch 'storm') to produce the generated code. This "fork" is actually Thrift 7 with all the Java packages renamed to be `org.apache.thrift7`. Otherwise, it's identical to Thrift 7. This fork was done because of the lack of backwards compatibility in Thrift and the need for many people to use other versions of Thrift in their Storm topologies.
+Every spout or bolt in a topology is given a user-specified identifier called the "component id". The component id is used to specify subscriptions from a bolt to the output streams of other spouts or bolts. A [StormTopology]({{page.git-blob-base}}/storm-client/src/storm.thrift) structure contains a map from component id to component for each type of component (spouts and bolts).
 
-Every spout or bolt in a topology is given a user-specified identifier called the "component id". The component id is used to specify subscriptions from a bolt to the output streams of other spouts or bolts. A [StormTopology]({{page.git-blob-base}}/storm-core/src/storm.thrift#L91) structure contains a map from component id to component for each type of component (spouts and bolts).
-
-Spouts and bolts have the same Thrift definition, so let's just take a look at the [Thrift definition for bolts]({{page.git-blob-base}}/storm-core/src/storm.thrift#L102). It contains a `ComponentObject` struct and a `ComponentCommon` struct.
+Spouts and bolts have the same Thrift definition, so let's just take a look at the [Thrift definition for bolts]({{page.git-blob-base}}/storm-client/src/storm.thrift). It contains a `ComponentObject` struct and a `ComponentCommon` struct.
 
 The `ComponentObject` defines the implementation for the bolt. It can be one of three types:
 
-1. A serialized java object (that implements [IBolt]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/task/IBolt.java))
-2. A `ShellComponent` object that indicates the implementation is in another language. Specifying a bolt this way will cause Storm to instantiate a [ShellBolt]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/task/ShellBolt.java) object to handle the communication between the JVM-based worker process and the non-JVM-based implementation of the component.
+1. A serialized java object (that implements [IBolt]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/task/IBolt.java))
+2. A `ShellComponent` object that indicates the implementation is in another language. Specifying a bolt this way will cause Storm to instantiate a [ShellBolt]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/task/ShellBolt.java) object to handle the communication between the JVM-based worker process and the non-JVM-based implementation of the component.
 3. A `JavaObject` structure which tells Storm the classname and constructor arguments to use to instantiate that bolt. This is useful if you want to define a topology in a non-JVM language. This way, you can make use of JVM-based spouts and bolts without having to create and serialize a Java object yourself.
 
 `ComponentCommon` defines everything else for this component. This includes:
@@ -36,7 +34,7 @@ The `ComponentObject` defines the implementation for the bolt. It can be one of
 3. The parallelism for this component
 4. The component-specific [configuration](Configuration.html) for this component
 
-Note that the structure spouts also have a `ComponentCommon` field, and so spouts can also have declarations to consume other input streams. Yet the Storm Java API does not provide a way for spouts to consume other streams, and if you put any input declarations there for a spout you would get an error when you tried to submit the topology. The reason that spouts have an input declarations field is not for users to use, but for Storm itself to use. Storm adds implicit streams and bolts to the topology to set up the [acking framework](Acking-framework-implementation.html), and two of these implicit streams are from the acker bolt to each spout in the topology. The acker sends "ack" or "fail" messages along these streams whenever a tuple tree is detected to be completed or failed. The code that transforms the user's topology into the runtime topology is located [here]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/common.clj#L279).
+Note that the structure spouts also have a `ComponentCommon` field, and so spouts can also have declarations to consume other input streams. Yet the Storm Java API does not provide a way for spouts to consume other streams, and if you put any input declarations there for a spout you would get an error when you tried to submit the topology. The reason that spouts have an input declarations field is not for users to use, but for Storm itself to use. Storm adds implicit streams and bolts to the topology to set up the [acking framework](Acking-framework-implementation.html), and two of these implicit streams are from the acker bolt to each spout in the topology. The acker sends "ack" or "fail" messages along these streams whenever a tuple tree is detected to be completed or failed. The code that transforms the user's topology into the runtime topology is located [here]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java).
 
 ### Java interfaces
 
@@ -55,80 +53,78 @@ You can see this strategy at work with the [BaseRichSpout](javadocs/org/apache/s
 
 Spouts and bolts are serialized into the Thrift definition of the topology as described above. 
 
-One subtle aspect of the interfaces is the difference between `IBolt` and `ISpout` vs. `IRichBolt` and `IRichSpout`. The main difference between them is the addition of the `declareOutputFields` method in the "Rich" versions of the interfaces. The reason for the split is that the output fields declaration for each output stream needs to be part of the Thrift struct (so it can be specified from any language), but as a user you want to be able to declare the streams as part of your class. What `TopologyBuilder` does when constructing the Thrift representation is call `declareOutputFields` to get the declaration and convert it into the Thrift structure. The conversion happens [at this portion]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java#L205) of the `TopologyBuilder` code.
+One subtle aspect of the interfaces is the difference between `IBolt` and `ISpout` vs. `IRichBolt` and `IRichSpout`. The main difference between them is the addition of the `declareOutputFields` method in the "Rich" versions of the interfaces. The reason for the split is that the output fields declaration for each output stream needs to be part of the Thrift struct (so it can be specified from any language), but as a user you want to be able to declare the streams as part of your class. What `TopologyBuilder` does when constructing the Thrift representation is call `declareOutputFields` to get the declaration and convert it into the Thrift structure. The conversion happens [at this portion]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java) of the `TopologyBuilder` code.
 
 
 ### Implementation
 
 Specifying all the functionality via Java interfaces ensures that every feature of Storm is available via Java. Moreso, the focus on Java interfaces ensures that the user experience from Java-land is pleasant as well.
 
-The implementation of Storm, on the other hand, is primarily in Clojure. While the codebase is about 50% Java and 50% Clojure in terms of LOC, most of the implementation logic is in Clojure. There are two notable exceptions to this, and that is the [DRPC](https://github.com/apache/storm/wiki/Distributed-RPC) and [transactional topologies](https://github.com/apache/storm/wiki/Transactional-topologies) implementations. These are implemented purely in Java. This was done to serve as an illustration for how to implement a higher level abstraction on Storm. The DRPC and transactional topologies implementations are in the [org.apache.storm.coordination]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/coordination), [org.apache.storm.drpc]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/drpc), and [org.apache.storm.transactional]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/transactional) packages.
+The implementation of Storm, on the other hand, is primarily in Clojure. While the codebase is about 50% Java and 50% Clojure in terms of LOC, most of the implementation logic is in Clojure. There are two notable exceptions to this, and that is the [DRPC](https://github.com/apache/storm/wiki/Distributed-RPC) and [transactional topologies](https://github.com/apache/storm/wiki/Transactional-topologies) implementations. These are implemented purely in Java. This was done to serve as an illustration for how to implement a higher level abstraction on Storm. The DRPC and transactional topologies implementations are in the [org.apache.storm.coordination]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/coordination), [org.apache.storm.drpc]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/drpc), and [org.apache.storm.transactional]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/transactional) packages.
 
 Here's a summary of the purpose of the main Java packages and Clojure namespace:
 
 #### Java packages
 
-[org.apache.storm.coordination]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/coordination): Implements the pieces required to coordinate batch-processing on top of Storm, which both DRPC and transactional topologies use. `CoordinatedBolt` is the most important class here.
+[org.apache.storm.coordination]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/coordination): Implements the pieces required to coordinate batch-processing on top of Storm, which both DRPC and transactional topologies use. `CoordinatedBolt` is the most important class here.
 
-[org.apache.storm.drpc]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/drpc): Implementation of the DRPC higher level abstraction
+[org.apache.storm.drpc]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/drpc): Implementation of the DRPC higher level abstraction
 
-[org.apache.storm.generated]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/generated): The generated Thrift code for Storm (generated using [this fork](https://github.com/nathanmarz/thrift) of Thrift, which simply renames the packages to org.apache.thrift7 to avoid conflicts with other Thrift versions)
+[org.apache.storm.generated]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/generated): The generated Thrift code for Storm (generated using [this fork](https://github.com/nathanmarz/thrift) of Thrift, which simply renames the packages to org.apache.thrift7 to avoid conflicts with other Thrift versions)
 
-[org.apache.storm.grouping]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/grouping): Contains interface for making custom stream groupings
+[org.apache.storm.grouping]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/grouping): Contains interface for making custom stream groupings
 
-[org.apache.storm.hooks]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/hooks): Interfaces for hooking into various events in Storm, such as when tasks emit tuples, when tuples are acked, etc. User guide for hooks is [here](https://github.com/apache/storm/wiki/Hooks).
+[org.apache.storm.hooks]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/hooks): Interfaces for hooking into various events in Storm, such as when tasks emit tuples, when tuples are acked, etc. User guide for hooks is [here](https://github.com/apache/storm/wiki/Hooks).
 
-[org.apache.storm.serialization]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/serialization): Implementation of how Storm serializes/deserializes tuples. Built on top of [Kryo](http://code.google.com/p/kryo/).
+[org.apache.storm.serialization]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/serialization): Implementation of how Storm serializes/deserializes tuples. Built on top of [Kryo](http://code.google.com/p/kryo/).
 
-[org.apache.storm.spout]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/spout): Definition of spout and associated interfaces (like the `SpoutOutputCollector`). Also contains `ShellSpout` which implements the protocol for defining spouts in non-JVM languages.
+[org.apache.storm.spout]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/spout): Definition of spout and associated interfaces (like the `SpoutOutputCollector`). Also contains `ShellSpout` which implements the protocol for defining spouts in non-JVM languages.
 
-[org.apache.storm.task]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/task): Definition of bolt and associated interfaces (like `OutputCollector`). Also contains `ShellBolt` which implements the protocol for defining bolts in non-JVM languages. Finally, `TopologyContext` is defined here as well, which is provided to spouts and bolts so they can get data about the topology and its execution at runtime.
+[org.apache.storm.task]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/task): Definition of bolt and associated interfaces (like `OutputCollector`). Also contains `ShellBolt` which implements the protocol for defining bolts in non-JVM languages. Finally, `TopologyContext` is defined here as well, which is provided to spouts and bolts so they can get data about the topology and its execution at runtime.
 
-[org.apache.storm.testing]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/testing): Contains a variety of test bolts and utilities used in Storm's unit tests.
+[org.apache.storm.testing]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/testing): Contains a variety of test bolts and utilities used in Storm's unit tests.
 
-[org.apache.storm.topology]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/topology): Java layer over the underlying Thrift structure to provide a clean, pure-Java API to Storm (users don't have to know about Thrift). `TopologyBuilder` is here as well as the helpful base classes for the different spouts and bolts. The slightly-higher level `IBasicBolt` interface is here, which is a simpler way to write certain kinds of bolts.
+[org.apache.storm.topology]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/topology): Java layer over the underlying Thrift structure to provide a clean, pure-Java API to Storm (users don't have to know about Thrift). `TopologyBuilder` is here as well as the helpful base classes for the different spouts and bolts. The slightly-higher level `IBasicBolt` interface is here, which is a simpler way to write certain kinds of bolts.
 
-[org.apache.storm.transactional]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/transactional): Implementation of transactional topologies.
+[org.apache.storm.transactional]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/transactional): Implementation of transactional topologies.
 
-[org.apache.storm.tuple]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/tuple): Implementation of Storm's tuple data model.
+[org.apache.storm.tuple]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/tuple): Implementation of Storm's tuple data model.
 
-[org.apache.storm.utils]({{page.git-tree-base}}/storm-core/src/jvm/org/apache/storm/tuple): Data structures and miscellaneous utilities used throughout the codebase.
+[org.apache.storm.utils]({{page.git-tree-base}}/storm-client/src/jvm/org/apache/storm/utils): Data structures and miscellaneous utilities used throughout the codebase.
 
 [org.apache.storm.command.*]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/command): These implement various commands for the `storm` command line client. These implementations are very short.
-[org.apache.storm.cluster]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/cluster): This code manages how cluster state (like what tasks are running where, what spout/bolt each task runs as) is stored, typically in Zookeeper.
+[org.apache.storm.cluster]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/cluster): This code manages how cluster state (like what tasks are running where, what spout/bolt each task runs as) is stored, typically in Zookeeper.
 
-[org.apache.storm.daemon.Acker]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/daemon/Acker.java): Implementation of the "acker" bolt, which is a key part of how Storm guarantees data processing.
+[org.apache.storm.daemon.Acker]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/Acker.java): Implementation of the "acker" bolt, which is a key part of how Storm guarantees data processing.
 
-[org.apache.storm.daemon.DrpcServer]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/daemon/DrpcServer.java): Implementation of the DRPC server for use with DRPC topologies.
+[org.apache.storm.daemon.DrpcServer]({{page.git-blob-base}}/storm-webapp/src/jvm/org/apache/storm/daemon/DrpcServer.java): Implementation of the DRPC server for use with DRPC topologies.
 
-[org.apache.storm.event]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/event): Implements a simple asynchronous function executor. Used in various places in Nimbus and Supervisor to make functions execute in serial to avoid any race conditions.
+[org.apache.storm.event]({{page.git-blob-base}}/storm-server/src/jvm/org/apache/storm/event): Implements a simple asynchronous function executor. Used in various places in Nimbus and Supervisor to make functions execute in serial to avoid any race conditions.
 
-[org.apache.storm.messaging.*]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/messaging): Defines a higher level interface to implementing point to point messaging. In local mode Storm uses in-memory Java queues to do this; on a cluster, it uses Netty, but it is pluggable.
+[org.apache.storm.messaging.*]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/messaging): Defines a higher level interface to implementing point to point messaging. In local mode Storm uses in-memory Java queues to do this; on a cluster, it uses Netty, but it is pluggable.
 
-[org.apache.storm.stats]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/stats): Implementation of stats rollup routines used when sending stats to ZK for use by the UI. Does things like windowed and rolling aggregations at multiple granularities.
+[org.apache.storm.stats]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/stats): Implementation of stats rollup routines used when sending stats to ZK for use by the UI. Does things like windowed and rolling aggregations at multiple granularities.
 
-[org.apache.storm.Thrift]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/Thrift.java): Wrappers around the generated Thrift API to make working with Thrift structures more pleasant.
+[org.apache.storm.Thrift]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/Thrift.java): Wrappers around the generated Thrift API to make working with Thrift structures more pleasant.
 
-[org.apache.storm.StormTimer]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/StormTimer.java): Implementation of a background timer to execute functions in the future or on a recurring interval. Storm couldn't use the [Timer](http://docs.oracle.com/javase/1.4.2/docs/api/java/util/Timer.html) class because it needed integration with time simulation in order to be able to unit test Nimbus and the Supervisor.
+[org.apache.storm.StormTimer]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/StormTimer.java): Implementation of a background timer to execute functions in the future or on a recurring interval. Storm couldn't use the [Timer](http://docs.oracle.com/javase/1.4.2/docs/api/java/util/Timer.html) class because it needed integration with time simulation in order to be able to unit test Nimbus and the Supervisor.
 
-#### Clojure namespaces
+[org.apache.storm.daemon.nimbus]({{page.git-blob-base}}/storm-server/src/jvm/org/apache/storm/daemon/nimbus/Nimbus.java): Implementation of Nimbus.
 
-[org.apache.storm.clojure]({{page.git-blob-base}}/storm-clojure/src/clj/org/apache/storm/clojure.clj): Implementation of the Clojure DSL for Storm.
-
-[org.apache.storm.config]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/config.clj): Created clojure symbols for config names in [Config.java](javadocs/org/apache/storm/Config.html)
- 
-[org.apache.storm.daemon.common]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/common.clj): Implementation of common functions used in Storm daemons, like getting the id for a topology based on the name, mapping a user's topology into the one that actually executes (with implicit acking streams and acker bolt added - see `system-topology!` function), and definitions for the various heartbeat and other structures persisted by Storm.
+[org.apache.storm.daemon.supervisor]({{page.git-blob-base}}/storm-server/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java): Implementation of Supervisor.
 
-[org.apache.storm.daemon.nimbus]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj): Implementation of Nimbus.
+[org.apache.storm.daemon.task]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/Task.java): Implementation of an individual task for a spout or bolt. Handles message routing, serialization, stats collection for the UI, as well as the spout-specific and bolt-specific execution implementations.
 
-[org.apache.storm.daemon.supervisor]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj): Implementation of Supervisor.
+[org.apache.storm.daemon.worker]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java): Implementation of a worker process (which will contain many tasks within). Implements message transferring and task launching.
 
-[org.apache.storm.daemon.task]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/task.clj): Implementation of an individual task for a spout or bolt. Handles message routing, serialization, stats collection for the UI, as well as the spout-specific and bolt-specific execution implementations.
+#### Clojure namespaces
 
-[org.apache.storm.daemon.worker]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/worker.clj): Implementation of a worker process (which will contain many tasks within). Implements message transferring and task launching.
+[org.apache.storm.clojure]({{page.git-blob-base}}/storm-clojure/src/clj/org/apache/storm/clojure.clj): Implementation of the Clojure DSL for Storm.
 
+[org.apache.storm.config]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/config.clj): Created clojure symbols for config names in [Config.java](javadocs/org/apache/storm/Config.html)
+ 
 [org.apache.storm.log]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/log.clj): Defines the functions used to log messages to log4j.
 
-[org.apache.storm.testing]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/testing.clj): Implementation of facilities used to test Storm topologies. Includes time simulation, `complete-topology` for running a fixed set of tuples through a topology and capturing the output, tracker topologies for having fine grained control over detecting when a cluster is "idle", and other utilities.
+[org.apache.storm.testing]({{page.git-blob-base}}/storm-clojure/src/clj/org/apache/storm/testing.clj): Implementation of facilities used to test Storm topologies. Includes time simulation, `complete-topology` for running a fixed set of tuples through a topology and capturing the output, tracker topologies for having fine grained control over detecting when a cluster is "idle", and other utilities.
 
 [org.apache.storm.ui.*]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/ui): Implementation of Storm UI. Completely independent from rest of code base and uses the Nimbus Thrift API to get data.

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Trident-API-Overview.md
----------------------------------------------------------------------
diff --git a/docs/Trident-API-Overview.md b/docs/Trident-API-Overview.md
index 309127d..4505367 100644
--- a/docs/Trident-API-Overview.md
+++ b/docs/Trident-API-Overview.md
@@ -608,7 +608,7 @@ The groupBy operation repartitions the stream by doing a partitionBy on the spec
 
 ![Grouping](images/grouping.png)
 
-If you run aggregators on a grouped stream, the aggregation will be run within each group instead of against the whole batch. persistentAggregate can also be run on a GroupedStream, in which case the results will be stored in a [MapState]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/state/map/MapState.java) with the key being the grouping fields. You can read more about persistentAggregate in the [Trident state doc](Trident-state.html).
+If you run aggregators on a grouped stream, the aggregation will be run within each group instead of against the whole batch. persistentAggregate can also be run on a GroupedStream, in which case the results will be stored in a [MapState]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/state/map/MapState.java) with the key being the grouping fields. You can read more about persistentAggregate in the [Trident state doc](Trident-state.html).
 
 Like regular streams, aggregators on grouped streams can be chained.
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Trident-spouts.md
----------------------------------------------------------------------
diff --git a/docs/Trident-spouts.md b/docs/Trident-spouts.md
index e433c4e..978881d 100644
--- a/docs/Trident-spouts.md
+++ b/docs/Trident-spouts.md
@@ -34,10 +34,10 @@ Even while processing multiple batches simultaneously, Trident will order any st
 
 Here are the following spout APIs available:
 
-1. [ITridentSpout]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/spout/ITridentSpout.java): The most general API that can support transactional or opaque transactional semantics. Generally you'll use one of the partitioned flavors of this API rather than this one directly.
-2. [IBatchSpout]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/spout/IBatchSpout.java): A non-transactional spout that emits batches of tuples at a time
-3. [IPartitionedTridentSpout]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/spout/IPartitionedTridentSpout.java): A transactional spout that reads from a partitioned data source (like a cluster of Kafka servers)
-4. [IOpaquePartitionedTridentSpout]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/spout/IOpaquePartitionedTridentSpout.java): An opaque transactional spout that reads from a partitioned data source
+1. [ITridentSpout]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/spout/ITridentSpout.java): The most general API that can support transactional or opaque transactional semantics. Generally you'll use one of the partitioned flavors of this API rather than this one directly.
+2. [IBatchSpout]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/spout/IBatchSpout.java): A non-transactional spout that emits batches of tuples at a time
+3. [IPartitionedTridentSpout]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/spout/IPartitionedTridentSpout.java): A transactional spout that reads from a partitioned data source (like a cluster of Kafka servers)
+4. [IOpaquePartitionedTridentSpout]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/spout/IOpaquePartitionedTridentSpout.java): An opaque transactional spout that reads from a partitioned data source
 
 And, like mentioned in the beginning of this tutorial, you can use regular IRichSpout's as well.
  

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Trident-state.md
----------------------------------------------------------------------
diff --git a/docs/Trident-state.md b/docs/Trident-state.md
index bb5b1ee..a89dc3c 100644
--- a/docs/Trident-state.md
+++ b/docs/Trident-state.md
@@ -309,7 +309,7 @@ public interface Snapshottable<T> extends State {
 }
 ```
 
-[MemoryMapState]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/testing/MemoryMapState.java) and [MemcachedState](https://github.com/nathanmarz/trident-memcached/blob/{{page.version}}/src/jvm/trident/memcached/MemcachedState.java) each implement both of these interfaces.
+[MemoryMapState]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryMapState.java) and [MemcachedState](https://github.com/nathanmarz/trident-memcached/blob/{{page.version}}/src/jvm/trident/memcached/MemcachedState.java) each implement both of these interfaces.
 
 ## Implementing Map States
 
@@ -322,10 +322,10 @@ public interface IBackingMap<T> {
 }
 ```
 
-OpaqueMap's will call multiPut with [OpaqueValue]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/state/OpaqueValue.java)'s for the vals, TransactionalMap's will give [TransactionalValue]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/state/TransactionalValue.java)'s for the vals, and NonTransactionalMaps will just pass the objects from the topology through.
+OpaqueMap's will call multiPut with [OpaqueValue]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/state/OpaqueValue.java)'s for the vals, TransactionalMap's will give [TransactionalValue]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/state/TransactionalValue.java)'s for the vals, and NonTransactionalMaps will just pass the objects from the topology through.
 
-Trident also provides the [CachedMap]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/state/map/CachedMap.java) class to do automatic LRU caching of map key/vals.
+Trident also provides the [CachedMap]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedMap.java) class to do automatic LRU caching of map key/vals.
 
-Finally, Trident provides the [SnapshottableMap]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/trident/state/map/SnapshottableMap.java) class that turns a MapState into a Snapshottable object, by storing global aggregations into a fixed key.
+Finally, Trident provides the [SnapshottableMap]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/trident/state/map/SnapshottableMap.java) class that turns a MapState into a Snapshottable object, by storing global aggregations into a fixed key.
 
 Take a look at the implementation of [MemcachedState](https://github.com/nathanmarz/trident-memcached/blob/master/src/jvm/trident/memcached/MemcachedState.java) to see how all these utilities can be put together to make a high performance MapState implementation. MemcachedState allows you to choose between opaque transactional, transactional, and non-transactional semantics.

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/Windowing.md
----------------------------------------------------------------------
diff --git a/docs/Windowing.md b/docs/Windowing.md
index cc4dfe4..91a6509 100644
--- a/docs/Windowing.md
+++ b/docs/Windowing.md
@@ -146,7 +146,7 @@ public BaseWindowedBolt withTimestampField(String fieldName)
 ```
 
 The value for the above `fieldName` will be looked up from the incoming tuple and considered for windowing calculations. 
-If the field is not present in the tuple an exception will be thrown. Alternatively a [TimestampExtractor](../storm-core/src/jvm/org/apache/storm/windowing/TimestampExtractor.java) can be used to
+If the field is not present in the tuple an exception will be thrown. Alternatively a [TimestampExtractor](../storm-client/src/jvm/org/apache/storm/windowing/TimestampExtractor.java) can be used to
 derive a timestamp value from a tuple (e.g. extract timestamp from a nested field within the tuple).
 
 ```java

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/docs/storm-kinesis.md
----------------------------------------------------------------------
diff --git a/docs/storm-kinesis.md b/docs/storm-kinesis.md
index b23c10d..6fae97b 100644
--- a/docs/storm-kinesis.md
+++ b/docs/storm-kinesis.md
@@ -106,7 +106,7 @@ Aws sdk version that this was tested with is 1.10.77
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-elasticsearch-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-elasticsearch-examples/pom.xml b/examples/storm-elasticsearch-examples/pom.xml
index f853ec4..b351a2b 100644
--- a/examples/storm-elasticsearch-examples/pom.xml
+++ b/examples/storm-elasticsearch-examples/pom.xml
@@ -29,7 +29,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-hbase-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-hbase-examples/pom.xml b/examples/storm-hbase-examples/pom.xml
index 8b1c15c..3014486 100644
--- a/examples/storm-hbase-examples/pom.xml
+++ b/examples/storm-hbase-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-hdfs-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/pom.xml b/examples/storm-hdfs-examples/pom.xml
index 69b90ef..3d63529 100644
--- a/examples/storm-hdfs-examples/pom.xml
+++ b/examples/storm-hdfs-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
index b787c54..4eab557 100644
--- a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
+++ b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
@@ -29,8 +29,8 @@ import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.generated.ClusterSummary;
 import org.apache.storm.metric.LoggingMetricsConsumer;
 import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.topology.base.BaseRichBolt;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.tuple.Fields;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-hive-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-hive-examples/pom.xml b/examples/storm-hive-examples/pom.xml
index 6f3f6ba..0c074e8 100644
--- a/examples/storm-hive-examples/pom.xml
+++ b/examples/storm-hive-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-jdbc-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-jdbc-examples/pom.xml b/examples/storm-jdbc-examples/pom.xml
index 9d488b2..ab35c6e 100644
--- a/examples/storm-jdbc-examples/pom.xml
+++ b/examples/storm-jdbc-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-jms-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-jms-examples/pom.xml b/examples/storm-jms-examples/pom.xml
index 4d232f7..1e10fde 100644
--- a/examples/storm-jms-examples/pom.xml
+++ b/examples/storm-jms-examples/pom.xml
@@ -60,7 +60,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <!-- keep storm out of the jar-with-dependencies -->
             <scope>${provided.scope}</scope>
@@ -139,7 +139,7 @@
                 <dependencies>
                     <dependency>
                         <groupId>org.apache.storm</groupId>
-                        <artifactId>storm-core</artifactId>
+                        <artifactId>storm-server</artifactId>
                         <version>${project.version}</version>
                         <type>jar</type>
                     </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-kafka-client-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-client-examples/pom.xml b/examples/storm-kafka-client-examples/pom.xml
index 554818b..e319346 100644
--- a/examples/storm-kafka-client-examples/pom.xml
+++ b/examples/storm-kafka-client-examples/pom.xml
@@ -34,7 +34,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-kafka-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/pom.xml b/examples/storm-kafka-examples/pom.xml
index 65eeda6..8e89aec 100644
--- a/examples/storm-kafka-examples/pom.xml
+++ b/examples/storm-kafka-examples/pom.xml
@@ -35,7 +35,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
index f71e2df..b08fc96 100644
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
+++ b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
@@ -20,9 +20,9 @@ package org.apache.storm.kafka.trident;
 
 import org.apache.storm.LocalDRPC;
 import org.apache.storm.generated.DistributedRPC;
-import org.apache.storm.thrift.transport.TTransportException;
-import org.apache.storm.utils.DRPCClient;
+import org.apache.thrift.transport.TTransportException;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.DRPCClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-mongodb-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/pom.xml b/examples/storm-mongodb-examples/pom.xml
index 297a6e8..e4a2a92 100644
--- a/examples/storm-mongodb-examples/pom.xml
+++ b/examples/storm-mongodb-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/WordCounter.java
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/WordCounter.java b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/WordCounter.java
index efb2d89..bc1e797 100644
--- a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/WordCounter.java
+++ b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/WordCounter.java
@@ -28,8 +28,6 @@ import com.google.common.collect.Maps;
 
 import java.util.Map;
 
-import static org.apache.storm.utils.Utils.tuple;
-
 public class WordCounter implements IBasicBolt {
     private Map<String, Integer> wordCounter = Maps.newHashMap();
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-mqtt-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-mqtt-examples/pom.xml b/examples/storm-mqtt-examples/pom.xml
index ca2d03b..41ff820 100644
--- a/examples/storm-mqtt-examples/pom.xml
+++ b/examples/storm-mqtt-examples/pom.xml
@@ -34,7 +34,7 @@
   <dependencies>
    <dependency>
       <groupId>org.apache.storm</groupId>
-      <artifactId>storm-core</artifactId>
+      <artifactId>storm-server</artifactId>
       <version>${project.version}</version>
       <scope>${provided.scope}</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-opentsdb-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-opentsdb-examples/pom.xml b/examples/storm-opentsdb-examples/pom.xml
index 9ffd12f..9f550e6 100644
--- a/examples/storm-opentsdb-examples/pom.xml
+++ b/examples/storm-opentsdb-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-perf/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-perf/pom.xml b/examples/storm-perf/pom.xml
index 64ae889..260eb06 100644
--- a/examples/storm-perf/pom.xml
+++ b/examples/storm-perf/pom.xml
@@ -83,7 +83,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <!--
               Use "provided" scope to keep storm out of the jar-with-dependencies

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaHdfsTopo.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaHdfsTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaHdfsTopo.java
index 4293aac..1dda177 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaHdfsTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaHdfsTopo.java
@@ -36,6 +36,7 @@ import org.apache.storm.perf.utils.Helper;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
 
 import java.util.Map;
 import java.util.UUID;
@@ -117,7 +118,7 @@ public class KafkaHdfsTopo {
 
 
   public static int getInt(Map map, Object key, int def) {
-    return Utils.getInt(Utils.get(map, key, def));
+    return ObjectReader.getInt(Utils.get(map, key, def));
   }
 
   public static String getStr(Map map, Object key) {

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaSpoutNullBoltTopo.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaSpoutNullBoltTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaSpoutNullBoltTopo.java
index 3512c65..e755ec5 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaSpoutNullBoltTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaSpoutNullBoltTopo.java
@@ -28,6 +28,7 @@ import org.apache.storm.perf.bolt.DevNullBolt;
 import org.apache.storm.perf.utils.Helper;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
 
 import java.util.Map;
 import java.util.UUID;
@@ -89,7 +90,7 @@ public class KafkaSpoutNullBoltTopo {
 
 
     public static int getInt(Map map, Object key, int def) {
-        return Utils.getInt(Utils.get(map, key, def));
+        return ObjectReader.getInt(Utils.get(map, key, def));
     }
 
     public static String getStr(Map map, Object key) {

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
index 686f9da..e01de0e 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
@@ -20,7 +20,6 @@ package org.apache.storm.perf.utils;
 
 import org.apache.storm.LocalCluster;
 import org.apache.storm.generated.Nimbus;
-import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
index 8bcd84f..465ff33 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
@@ -24,8 +24,9 @@ import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.KillOptions;
 import org.apache.storm.generated.Nimbus;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.NimbusClient;
 
 import java.util.Map;
 
@@ -53,7 +54,7 @@ public class Helper {
     }
 
     public static int getInt(Map map, Object key, int def) {
-        return Utils.getInt(Utils.get(map, key, def));
+        return ObjectReader.getInt(Utils.get(map, key, def));
     }
 
     public static String getStr(Map map, Object key) {

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-pmml-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-pmml-examples/pom.xml b/examples/storm-pmml-examples/pom.xml
index 05ec99f..d202076 100644
--- a/examples/storm-pmml-examples/pom.xml
+++ b/examples/storm-pmml-examples/pom.xml
@@ -34,7 +34,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-redis-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/pom.xml b/examples/storm-redis-examples/pom.xml
index 3532237..9317daf 100644
--- a/examples/storm-redis-examples/pom.xml
+++ b/examples/storm-redis-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WordCounter.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WordCounter.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WordCounter.java
index 6fa930c..2dd95ff 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WordCounter.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WordCounter.java
@@ -28,8 +28,6 @@ import com.google.common.collect.Maps;
 
 import java.util.Map;
 
-import static org.apache.storm.utils.Utils.tuple;
-
 public class WordCounter implements IBasicBolt {
     private Map<String, Integer> wordCounter = Maps.newHashMap();
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-solr-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-solr-examples/pom.xml b/examples/storm-solr-examples/pom.xml
index 1cfa796..d6d86fc 100644
--- a/examples/storm-solr-examples/pom.xml
+++ b/examples/storm-solr-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
+            <artifactId>storm-server</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/README.markdown
----------------------------------------------------------------------
diff --git a/examples/storm-starter/README.markdown b/examples/storm-starter/README.markdown
index f8693a1..6a88465 100644
--- a/examples/storm-starter/README.markdown
+++ b/examples/storm-starter/README.markdown
@@ -99,7 +99,7 @@ With submitting you can run topologies which use multilang, for example, `WordCo
 _Submitting a topology in local vs. remote mode:_
 It depends on the actual code of a topology how you can or even must tell Storm whether to run the topology locally (in
 an in-memory LocalCluster instance of Storm) or remotely (in a "real" Storm cluster).  In the case of topologies extending
-[ConfigurableTopology](https://github.com/apache/storm/tree/master/storm-core/src/jvm/org/apache/storm/topology/ConfigurableTopology.java),
+[ConfigurableTopology](https://github.com/apache/storm/tree/master/storm-server/src/jvm/org/apache/storm/topology/ConfigurableTopology.java),
 such as [RollingTopWords](src/jvm/org/apache/storm/starter/RollingTopWords.java) or [ExclamationTopology](src/jvm/org/apache/storm/starter/ExclamationTopology.java), 
 this is done by specifying `-local` in the command line arguments.
 Other topologies may behave differently, e.g. by always submitting to a remote cluster (i.e. hardcoded in a way that you, as a user,

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index 69acdb8..7d102e7 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -77,6 +77,11 @@
         -->
         <scope>${provided.scope}</scope>
     </dependency>
+      <!--
+      normally including 'storm-server' is OK for LocalCluster,
+      but 'storm-starter' also uses clojure implementation
+      so 'storm-starter' needs to include 'storm-core'
+      -->
     <dependency>
       <groupId>org.apache.storm</groupId>
       <artifactId>storm-core</artifactId>
@@ -89,7 +94,13 @@
     </dependency>
       <dependency>
           <groupId>org.apache.storm</groupId>
-          <artifactId>storm-core</artifactId>
+          <artifactId>storm-client-misc</artifactId>
+          <version>${project.version}</version>
+          <scope>${provided.scope}</scope>
+      </dependency>
+      <dependency>
+          <groupId>org.apache.storm</groupId>
+          <artifactId>storm-client</artifactId>
           <version>${project.version}</version>
           <type>test-jar</type>
           <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
index eed1a91..6bcd35c 100644
--- a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
+++ b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
@@ -15,7 +15,7 @@
 ;; limitations under the License.
 (ns org.apache.storm.starter.clj.exclamation
   (:import [org.apache.storm StormSubmitter LocalCluster]
-           [org.apache.storm.utils Utils]
+    [org.apache.storm.utils Utils]
            [org.apache.storm.testing TestWordSpout])
   (:use [org.apache.storm clojure config])
   (:gen-class))
@@ -32,9 +32,9 @@
     "exclaim2" (bolt-spec {"exclaim1" :shuffle} exclamation-bolt :p 2)}))
 
 (defn run-local! []
-  (with-open [cluster (LocalCluster.)
+      (with-open [cluster (LocalCluster.)
               topo (.submitTopology cluster "exclamation" {TOPOLOGY-DEBUG true} (mk-topology))]
-    (Utils/sleep 10000)))
+                 (Utils/sleep 10000)))
 
 (defn submit-topology! [name]
   (StormSubmitter/submitTopologyWithProgressBar

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
index 937ef70..d5eada5 100644
--- a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
+++ b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
@@ -18,8 +18,8 @@
             [org.apache.storm.starter.clj.bolts :refer
              [rolling-count-bolt intermediate-rankings-bolt total-rankings-bolt]])
   (:import [org.apache.storm StormSubmitter LocalCluster]
-           [org.apache.storm.utils Utils]
-           [org.apache.storm.testing TestWordSpout])
+    [org.apache.storm.utils Utils]
+    [org.apache.storm.testing TestWordSpout])
   (:gen-class))
 
 (defn mk-topology []
@@ -39,9 +39,9 @@
                                  (total-rankings-bolt 5 2))})))
 
 (defn run-local! []
-  (with-open [cluster (LocalCluster.)
+      (with-open [cluster (LocalCluster.)
               topo (.submitTopology cluster "slidingWindowCounts" {TOPOLOGY-DEBUG true} (mk-topology))]
-    (Utils/sleep 60000)))
+                 (Utils/sleep 60000)))
 
 (defn submit-topology! [name]
   (StormSubmitter/submitTopology

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
index 13ccb1d..a6352c9 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
@@ -21,11 +21,8 @@ import org.apache.storm.Config;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.blobstore.AtomicOutputStream;
 import org.apache.storm.blobstore.ClientBlobStore;
-import org.apache.storm.blobstore.InputStreamWithMeta;
-import org.apache.storm.blobstore.NimbusBlobStore;
 
 import org.apache.storm.generated.AccessControl;
-import org.apache.storm.generated.AccessControlType;
 import org.apache.storm.generated.AlreadyAliveException;
 import org.apache.storm.generated.AuthorizationException;
 import org.apache.storm.generated.InvalidTopologyException;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
index 5d86ff0..5acf908 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
@@ -23,7 +23,6 @@ import org.apache.storm.generated.*;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.IRichBolt;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseBasicBolt;
@@ -31,8 +30,8 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.NimbusClient;
 
 import java.util.HashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
index 1684ce5..78907ea 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
@@ -24,7 +24,6 @@ import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.BasicOutputCollector;
 import org.apache.storm.topology.FailedException;
-import org.apache.storm.topology.IRichBolt;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseBasicBolt;
@@ -32,12 +31,11 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.NimbusClient;
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Random;
 
 public class InOrderDeliveryTest {
   public static class InOrderSpout extends BaseRichSpout {

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
index 17cabf5..aca5c7b 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
@@ -35,7 +35,6 @@ import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.List;
 import java.util.Map;
 
 /**

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java
new file mode 100644
index 0000000..cd8fa2c
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.starter.spout.RandomIntegerSpout;
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.topology.base.BaseStatefulBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * An example topology that demonstrates the use of {@link org.apache.storm.topology.IStatefulBolt}
+ * to manage state. To run the example,
+ * <pre>
+ * $ storm jar examples/storm-starter/storm-starter-topologies-*.jar storm.starter.StatefulTopology statetopology
+ * </pre>
+ * <p/>
+ * The default state used is 'InMemoryKeyValueState' which does not persist the state across restarts. You could use
+ * 'RedisKeyValueState' to test state persistence by setting below property in conf/storm.yaml
+ * <pre>
+ * topology.state.provider: org.apache.storm.redis.state.RedisKeyValueStateProvider
+ * </pre>
+ * <p/>
+ * You should also start a local redis instance before running the 'storm jar' command. The default
+ * RedisKeyValueStateProvider parameters can be overridden in conf/storm.yaml, for e.g.
+ * <p/>
+ * <pre>
+ * topology.state.provider.config: '{"keyClass":"...", "valueClass":"...",
+ *                                   "keySerializerClass":"...", "valueSerializerClass":"...",
+ *                                   "jedisPoolConfig":{"host":"localhost", "port":6379,
+ *                                      "timeout":2000, "database":0, "password":"xyz"}}'
+ *
+ * </pre>
+ * </p>
+ */
+public class StatefulTopology {
+    private static final Logger LOG = LoggerFactory.getLogger(StatefulTopology.class);
+    /**
+     * A bolt that uses {@link KeyValueState} to save its state.
+     */
+    private static class StatefulSumBolt extends BaseStatefulBolt<KeyValueState<String, Long>> {
+        String name;
+        KeyValueState<String, Long> kvState;
+        long sum;
+        private OutputCollector collector;
+
+        StatefulSumBolt(String name) {
+            this.name = name;
+        }
+
+        @Override
+        public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+            this.collector = collector;
+        }
+
+        @Override
+        public void execute(Tuple input) {
+            sum += ((Number) input.getValueByField("value")).longValue();
+            LOG.debug("{} sum = {}", name, sum);
+            kvState.put("sum", sum);
+            collector.emit(input, new Values(sum));
+            collector.ack(input);
+        }
+
+        @Override
+        public void initState(KeyValueState<String, Long> state) {
+            kvState = state;
+            sum = kvState.get("sum", 0L);
+            LOG.debug("Initstate, sum from saved state = {} ", sum);
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("value"));
+        }
+    }
+
+    public static class PrinterBolt extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            System.out.println(tuple);
+            LOG.debug("Got tuple {}", tuple);
+            collector.emit(tuple.getValues());
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer ofd) {
+            ofd.declare(new Fields("value"));
+        }
+
+    }
+
+    public static void main(String[] args) throws Exception {
+        TopologyBuilder builder = new TopologyBuilder();
+        builder.setSpout("spout", new RandomIntegerSpout());
+        builder.setBolt("partialsum", new StatefulSumBolt("partial"), 1).shuffleGrouping("spout");
+        builder.setBolt("printer", new PrinterBolt(), 2).shuffleGrouping("partialsum");
+        builder.setBolt("total", new StatefulSumBolt("total"), 1).shuffleGrouping("printer");
+        Config conf = new Config();
+        conf.setDebug(false);
+
+        if (args != null && args.length > 0) {
+            conf.setNumWorkers(1);
+            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+        } else {
+            try (LocalCluster cluster = new LocalCluster();
+                 LocalTopology topology = cluster.submitTopology("test", conf, builder.createTopology());) {
+                Utils.sleep(40000);
+            }
+        }
+    }
+}


[15/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SpoutStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SpoutStats.java b/storm-client/src/jvm/org/apache/storm/generated/SpoutStats.java
new file mode 100644
index 0000000..b1384f6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SpoutStats.java
@@ -0,0 +1,917 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStats._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutStats");
+
+  private static final org.apache.thrift.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("acked", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("failed", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField COMPLETE_MS_AVG_FIELD_DESC = new org.apache.thrift.protocol.TField("complete_ms_avg", org.apache.thrift.protocol.TType.MAP, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SpoutStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SpoutStatsTupleSchemeFactory());
+  }
+
+  private Map<String,Map<String,Long>> acked; // required
+  private Map<String,Map<String,Long>> failed; // required
+  private Map<String,Map<String,Double>> complete_ms_avg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ACKED((short)1, "acked"),
+    FAILED((short)2, "failed"),
+    COMPLETE_MS_AVG((short)3, "complete_ms_avg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACKED
+          return ACKED;
+        case 2: // FAILED
+          return FAILED;
+        case 3: // COMPLETE_MS_AVG
+          return COMPLETE_MS_AVG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACKED, new org.apache.thrift.meta_data.FieldMetaData("acked", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
+    tmpMap.put(_Fields.FAILED, new org.apache.thrift.meta_data.FieldMetaData("failed", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
+    tmpMap.put(_Fields.COMPLETE_MS_AVG, new org.apache.thrift.meta_data.FieldMetaData("complete_ms_avg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpoutStats.class, metaDataMap);
+  }
+
+  public SpoutStats() {
+  }
+
+  public SpoutStats(
+    Map<String,Map<String,Long>> acked,
+    Map<String,Map<String,Long>> failed,
+    Map<String,Map<String,Double>> complete_ms_avg)
+  {
+    this();
+    this.acked = acked;
+    this.failed = failed;
+    this.complete_ms_avg = complete_ms_avg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SpoutStats(SpoutStats other) {
+    if (other.is_set_acked()) {
+      Map<String,Map<String,Long>> __this__acked = new HashMap<String,Map<String,Long>>(other.acked.size());
+      for (Map.Entry<String, Map<String,Long>> other_element : other.acked.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<String,Long> other_element_value = other_element.getValue();
+
+        String __this__acked_copy_key = other_element_key;
+
+        Map<String,Long> __this__acked_copy_value = new HashMap<String,Long>(other_element_value);
+
+        __this__acked.put(__this__acked_copy_key, __this__acked_copy_value);
+      }
+      this.acked = __this__acked;
+    }
+    if (other.is_set_failed()) {
+      Map<String,Map<String,Long>> __this__failed = new HashMap<String,Map<String,Long>>(other.failed.size());
+      for (Map.Entry<String, Map<String,Long>> other_element : other.failed.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<String,Long> other_element_value = other_element.getValue();
+
+        String __this__failed_copy_key = other_element_key;
+
+        Map<String,Long> __this__failed_copy_value = new HashMap<String,Long>(other_element_value);
+
+        __this__failed.put(__this__failed_copy_key, __this__failed_copy_value);
+      }
+      this.failed = __this__failed;
+    }
+    if (other.is_set_complete_ms_avg()) {
+      Map<String,Map<String,Double>> __this__complete_ms_avg = new HashMap<String,Map<String,Double>>(other.complete_ms_avg.size());
+      for (Map.Entry<String, Map<String,Double>> other_element : other.complete_ms_avg.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<String,Double> other_element_value = other_element.getValue();
+
+        String __this__complete_ms_avg_copy_key = other_element_key;
+
+        Map<String,Double> __this__complete_ms_avg_copy_value = new HashMap<String,Double>(other_element_value);
+
+        __this__complete_ms_avg.put(__this__complete_ms_avg_copy_key, __this__complete_ms_avg_copy_value);
+      }
+      this.complete_ms_avg = __this__complete_ms_avg;
+    }
+  }
+
+  public SpoutStats deepCopy() {
+    return new SpoutStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.acked = null;
+    this.failed = null;
+    this.complete_ms_avg = null;
+  }
+
+  public int get_acked_size() {
+    return (this.acked == null) ? 0 : this.acked.size();
+  }
+
+  public void put_to_acked(String key, Map<String,Long> val) {
+    if (this.acked == null) {
+      this.acked = new HashMap<String,Map<String,Long>>();
+    }
+    this.acked.put(key, val);
+  }
+
+  public Map<String,Map<String,Long>> get_acked() {
+    return this.acked;
+  }
+
+  public void set_acked(Map<String,Map<String,Long>> acked) {
+    this.acked = acked;
+  }
+
+  public void unset_acked() {
+    this.acked = null;
+  }
+
+  /** Returns true if field acked is set (has been assigned a value) and false otherwise */
+  public boolean is_set_acked() {
+    return this.acked != null;
+  }
+
+  public void set_acked_isSet(boolean value) {
+    if (!value) {
+      this.acked = null;
+    }
+  }
+
+  public int get_failed_size() {
+    return (this.failed == null) ? 0 : this.failed.size();
+  }
+
+  public void put_to_failed(String key, Map<String,Long> val) {
+    if (this.failed == null) {
+      this.failed = new HashMap<String,Map<String,Long>>();
+    }
+    this.failed.put(key, val);
+  }
+
+  public Map<String,Map<String,Long>> get_failed() {
+    return this.failed;
+  }
+
+  public void set_failed(Map<String,Map<String,Long>> failed) {
+    this.failed = failed;
+  }
+
+  public void unset_failed() {
+    this.failed = null;
+  }
+
+  /** Returns true if field failed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_failed() {
+    return this.failed != null;
+  }
+
+  public void set_failed_isSet(boolean value) {
+    if (!value) {
+      this.failed = null;
+    }
+  }
+
+  public int get_complete_ms_avg_size() {
+    return (this.complete_ms_avg == null) ? 0 : this.complete_ms_avg.size();
+  }
+
+  public void put_to_complete_ms_avg(String key, Map<String,Double> val) {
+    if (this.complete_ms_avg == null) {
+      this.complete_ms_avg = new HashMap<String,Map<String,Double>>();
+    }
+    this.complete_ms_avg.put(key, val);
+  }
+
+  public Map<String,Map<String,Double>> get_complete_ms_avg() {
+    return this.complete_ms_avg;
+  }
+
+  public void set_complete_ms_avg(Map<String,Map<String,Double>> complete_ms_avg) {
+    this.complete_ms_avg = complete_ms_avg;
+  }
+
+  public void unset_complete_ms_avg() {
+    this.complete_ms_avg = null;
+  }
+
+  /** Returns true if field complete_ms_avg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_complete_ms_avg() {
+    return this.complete_ms_avg != null;
+  }
+
+  public void set_complete_ms_avg_isSet(boolean value) {
+    if (!value) {
+      this.complete_ms_avg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACKED:
+      if (value == null) {
+        unset_acked();
+      } else {
+        set_acked((Map<String,Map<String,Long>>)value);
+      }
+      break;
+
+    case FAILED:
+      if (value == null) {
+        unset_failed();
+      } else {
+        set_failed((Map<String,Map<String,Long>>)value);
+      }
+      break;
+
+    case COMPLETE_MS_AVG:
+      if (value == null) {
+        unset_complete_ms_avg();
+      } else {
+        set_complete_ms_avg((Map<String,Map<String,Double>>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACKED:
+      return get_acked();
+
+    case FAILED:
+      return get_failed();
+
+    case COMPLETE_MS_AVG:
+      return get_complete_ms_avg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ACKED:
+      return is_set_acked();
+    case FAILED:
+      return is_set_failed();
+    case COMPLETE_MS_AVG:
+      return is_set_complete_ms_avg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SpoutStats)
+      return this.equals((SpoutStats)that);
+    return false;
+  }
+
+  public boolean equals(SpoutStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_acked = true && this.is_set_acked();
+    boolean that_present_acked = true && that.is_set_acked();
+    if (this_present_acked || that_present_acked) {
+      if (!(this_present_acked && that_present_acked))
+        return false;
+      if (!this.acked.equals(that.acked))
+        return false;
+    }
+
+    boolean this_present_failed = true && this.is_set_failed();
+    boolean that_present_failed = true && that.is_set_failed();
+    if (this_present_failed || that_present_failed) {
+      if (!(this_present_failed && that_present_failed))
+        return false;
+      if (!this.failed.equals(that.failed))
+        return false;
+    }
+
+    boolean this_present_complete_ms_avg = true && this.is_set_complete_ms_avg();
+    boolean that_present_complete_ms_avg = true && that.is_set_complete_ms_avg();
+    if (this_present_complete_ms_avg || that_present_complete_ms_avg) {
+      if (!(this_present_complete_ms_avg && that_present_complete_ms_avg))
+        return false;
+      if (!this.complete_ms_avg.equals(that.complete_ms_avg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_acked = true && (is_set_acked());
+    list.add(present_acked);
+    if (present_acked)
+      list.add(acked);
+
+    boolean present_failed = true && (is_set_failed());
+    list.add(present_failed);
+    if (present_failed)
+      list.add(failed);
+
+    boolean present_complete_ms_avg = true && (is_set_complete_ms_avg());
+    list.add(present_complete_ms_avg);
+    if (present_complete_ms_avg)
+      list.add(complete_ms_avg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SpoutStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_acked()).compareTo(other.is_set_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acked()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acked, other.acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_failed()).compareTo(other.is_set_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_failed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.failed, other.failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_complete_ms_avg()).compareTo(other.is_set_complete_ms_avg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_complete_ms_avg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.complete_ms_avg, other.complete_ms_avg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SpoutStats(");
+    boolean first = true;
+
+    sb.append("acked:");
+    if (this.acked == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.acked);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("failed:");
+    if (this.failed == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.failed);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("complete_ms_avg:");
+    if (this.complete_ms_avg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.complete_ms_avg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_acked()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'acked' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_failed()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'failed' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_complete_ms_avg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'complete_ms_avg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SpoutStatsStandardSchemeFactory implements SchemeFactory {
+    public SpoutStatsStandardScheme getScheme() {
+      return new SpoutStatsStandardScheme();
+    }
+  }
+
+  private static class SpoutStatsStandardScheme extends StandardScheme<SpoutStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SpoutStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ACKED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map232 = iprot.readMapBegin();
+                struct.acked = new HashMap<String,Map<String,Long>>(2*_map232.size);
+                String _key233;
+                Map<String,Long> _val234;
+                for (int _i235 = 0; _i235 < _map232.size; ++_i235)
+                {
+                  _key233 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map236 = iprot.readMapBegin();
+                    _val234 = new HashMap<String,Long>(2*_map236.size);
+                    String _key237;
+                    long _val238;
+                    for (int _i239 = 0; _i239 < _map236.size; ++_i239)
+                    {
+                      _key237 = iprot.readString();
+                      _val238 = iprot.readI64();
+                      _val234.put(_key237, _val238);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.acked.put(_key233, _val234);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_acked_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // FAILED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map240 = iprot.readMapBegin();
+                struct.failed = new HashMap<String,Map<String,Long>>(2*_map240.size);
+                String _key241;
+                Map<String,Long> _val242;
+                for (int _i243 = 0; _i243 < _map240.size; ++_i243)
+                {
+                  _key241 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map244 = iprot.readMapBegin();
+                    _val242 = new HashMap<String,Long>(2*_map244.size);
+                    String _key245;
+                    long _val246;
+                    for (int _i247 = 0; _i247 < _map244.size; ++_i247)
+                    {
+                      _key245 = iprot.readString();
+                      _val246 = iprot.readI64();
+                      _val242.put(_key245, _val246);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.failed.put(_key241, _val242);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_failed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // COMPLETE_MS_AVG
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map248 = iprot.readMapBegin();
+                struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map248.size);
+                String _key249;
+                Map<String,Double> _val250;
+                for (int _i251 = 0; _i251 < _map248.size; ++_i251)
+                {
+                  _key249 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map252 = iprot.readMapBegin();
+                    _val250 = new HashMap<String,Double>(2*_map252.size);
+                    String _key253;
+                    double _val254;
+                    for (int _i255 = 0; _i255 < _map252.size; ++_i255)
+                    {
+                      _key253 = iprot.readString();
+                      _val254 = iprot.readDouble();
+                      _val250.put(_key253, _val254);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.complete_ms_avg.put(_key249, _val250);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_complete_ms_avg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SpoutStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.acked != null) {
+        oprot.writeFieldBegin(ACKED_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
+          for (Map.Entry<String, Map<String,Long>> _iter256 : struct.acked.entrySet())
+          {
+            oprot.writeString(_iter256.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter256.getValue().size()));
+              for (Map.Entry<String, Long> _iter257 : _iter256.getValue().entrySet())
+              {
+                oprot.writeString(_iter257.getKey());
+                oprot.writeI64(_iter257.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.failed != null) {
+        oprot.writeFieldBegin(FAILED_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
+          for (Map.Entry<String, Map<String,Long>> _iter258 : struct.failed.entrySet())
+          {
+            oprot.writeString(_iter258.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter258.getValue().size()));
+              for (Map.Entry<String, Long> _iter259 : _iter258.getValue().entrySet())
+              {
+                oprot.writeString(_iter259.getKey());
+                oprot.writeI64(_iter259.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.complete_ms_avg != null) {
+        oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.complete_ms_avg.size()));
+          for (Map.Entry<String, Map<String,Double>> _iter260 : struct.complete_ms_avg.entrySet())
+          {
+            oprot.writeString(_iter260.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, _iter260.getValue().size()));
+              for (Map.Entry<String, Double> _iter261 : _iter260.getValue().entrySet())
+              {
+                oprot.writeString(_iter261.getKey());
+                oprot.writeDouble(_iter261.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SpoutStatsTupleSchemeFactory implements SchemeFactory {
+    public SpoutStatsTupleScheme getScheme() {
+      return new SpoutStatsTupleScheme();
+    }
+  }
+
+  private static class SpoutStatsTupleScheme extends TupleScheme<SpoutStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SpoutStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.acked.size());
+        for (Map.Entry<String, Map<String,Long>> _iter262 : struct.acked.entrySet())
+        {
+          oprot.writeString(_iter262.getKey());
+          {
+            oprot.writeI32(_iter262.getValue().size());
+            for (Map.Entry<String, Long> _iter263 : _iter262.getValue().entrySet())
+            {
+              oprot.writeString(_iter263.getKey());
+              oprot.writeI64(_iter263.getValue());
+            }
+          }
+        }
+      }
+      {
+        oprot.writeI32(struct.failed.size());
+        for (Map.Entry<String, Map<String,Long>> _iter264 : struct.failed.entrySet())
+        {
+          oprot.writeString(_iter264.getKey());
+          {
+            oprot.writeI32(_iter264.getValue().size());
+            for (Map.Entry<String, Long> _iter265 : _iter264.getValue().entrySet())
+            {
+              oprot.writeString(_iter265.getKey());
+              oprot.writeI64(_iter265.getValue());
+            }
+          }
+        }
+      }
+      {
+        oprot.writeI32(struct.complete_ms_avg.size());
+        for (Map.Entry<String, Map<String,Double>> _iter266 : struct.complete_ms_avg.entrySet())
+        {
+          oprot.writeString(_iter266.getKey());
+          {
+            oprot.writeI32(_iter266.getValue().size());
+            for (Map.Entry<String, Double> _iter267 : _iter266.getValue().entrySet())
+            {
+              oprot.writeString(_iter267.getKey());
+              oprot.writeDouble(_iter267.getValue());
+            }
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SpoutStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map268 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.acked = new HashMap<String,Map<String,Long>>(2*_map268.size);
+        String _key269;
+        Map<String,Long> _val270;
+        for (int _i271 = 0; _i271 < _map268.size; ++_i271)
+        {
+          _key269 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map272 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val270 = new HashMap<String,Long>(2*_map272.size);
+            String _key273;
+            long _val274;
+            for (int _i275 = 0; _i275 < _map272.size; ++_i275)
+            {
+              _key273 = iprot.readString();
+              _val274 = iprot.readI64();
+              _val270.put(_key273, _val274);
+            }
+          }
+          struct.acked.put(_key269, _val270);
+        }
+      }
+      struct.set_acked_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map276 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.failed = new HashMap<String,Map<String,Long>>(2*_map276.size);
+        String _key277;
+        Map<String,Long> _val278;
+        for (int _i279 = 0; _i279 < _map276.size; ++_i279)
+        {
+          _key277 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map280 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val278 = new HashMap<String,Long>(2*_map280.size);
+            String _key281;
+            long _val282;
+            for (int _i283 = 0; _i283 < _map280.size; ++_i283)
+            {
+              _key281 = iprot.readString();
+              _val282 = iprot.readI64();
+              _val278.put(_key281, _val282);
+            }
+          }
+          struct.failed.put(_key277, _val278);
+        }
+      }
+      struct.set_failed_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map284 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map284.size);
+        String _key285;
+        Map<String,Double> _val286;
+        for (int _i287 = 0; _i287 < _map284.size; ++_i287)
+        {
+          _key285 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map288 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val286 = new HashMap<String,Double>(2*_map288.size);
+            String _key289;
+            double _val290;
+            for (int _i291 = 0; _i291 < _map288.size; ++_i291)
+            {
+              _key289 = iprot.readString();
+              _val290 = iprot.readDouble();
+              _val286.put(_key289, _val290);
+            }
+          }
+          struct.complete_ms_avg.put(_key285, _val286);
+        }
+      }
+      struct.set_complete_ms_avg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/StateSpoutSpec.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/StateSpoutSpec.java b/storm-client/src/jvm/org/apache/storm/generated/StateSpoutSpec.java
new file mode 100644
index 0000000..c6ab89d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/StateSpoutSpec.java
@@ -0,0 +1,514 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class StateSpoutSpec implements org.apache.thrift.TBase<StateSpoutSpec, StateSpoutSpec._Fields>, java.io.Serializable, Cloneable, Comparable<StateSpoutSpec> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StateSpoutSpec");
+
+  private static final org.apache.thrift.protocol.TField STATE_SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("state_spout_object", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift.protocol.TField("common", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new StateSpoutSpecStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new StateSpoutSpecTupleSchemeFactory());
+  }
+
+  private ComponentObject state_spout_object; // required
+  private ComponentCommon common; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    STATE_SPOUT_OBJECT((short)1, "state_spout_object"),
+    COMMON((short)2, "common");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STATE_SPOUT_OBJECT
+          return STATE_SPOUT_OBJECT;
+        case 2: // COMMON
+          return COMMON;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STATE_SPOUT_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("state_spout_object", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentObject.class)));
+    tmpMap.put(_Fields.COMMON, new org.apache.thrift.meta_data.FieldMetaData("common", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentCommon.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StateSpoutSpec.class, metaDataMap);
+  }
+
+  public StateSpoutSpec() {
+  }
+
+  public StateSpoutSpec(
+    ComponentObject state_spout_object,
+    ComponentCommon common)
+  {
+    this();
+    this.state_spout_object = state_spout_object;
+    this.common = common;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StateSpoutSpec(StateSpoutSpec other) {
+    if (other.is_set_state_spout_object()) {
+      this.state_spout_object = new ComponentObject(other.state_spout_object);
+    }
+    if (other.is_set_common()) {
+      this.common = new ComponentCommon(other.common);
+    }
+  }
+
+  public StateSpoutSpec deepCopy() {
+    return new StateSpoutSpec(this);
+  }
+
+  @Override
+  public void clear() {
+    this.state_spout_object = null;
+    this.common = null;
+  }
+
+  public ComponentObject get_state_spout_object() {
+    return this.state_spout_object;
+  }
+
+  public void set_state_spout_object(ComponentObject state_spout_object) {
+    this.state_spout_object = state_spout_object;
+  }
+
+  public void unset_state_spout_object() {
+    this.state_spout_object = null;
+  }
+
+  /** Returns true if field state_spout_object is set (has been assigned a value) and false otherwise */
+  public boolean is_set_state_spout_object() {
+    return this.state_spout_object != null;
+  }
+
+  public void set_state_spout_object_isSet(boolean value) {
+    if (!value) {
+      this.state_spout_object = null;
+    }
+  }
+
+  public ComponentCommon get_common() {
+    return this.common;
+  }
+
+  public void set_common(ComponentCommon common) {
+    this.common = common;
+  }
+
+  public void unset_common() {
+    this.common = null;
+  }
+
+  /** Returns true if field common is set (has been assigned a value) and false otherwise */
+  public boolean is_set_common() {
+    return this.common != null;
+  }
+
+  public void set_common_isSet(boolean value) {
+    if (!value) {
+      this.common = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STATE_SPOUT_OBJECT:
+      if (value == null) {
+        unset_state_spout_object();
+      } else {
+        set_state_spout_object((ComponentObject)value);
+      }
+      break;
+
+    case COMMON:
+      if (value == null) {
+        unset_common();
+      } else {
+        set_common((ComponentCommon)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STATE_SPOUT_OBJECT:
+      return get_state_spout_object();
+
+    case COMMON:
+      return get_common();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case STATE_SPOUT_OBJECT:
+      return is_set_state_spout_object();
+    case COMMON:
+      return is_set_common();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StateSpoutSpec)
+      return this.equals((StateSpoutSpec)that);
+    return false;
+  }
+
+  public boolean equals(StateSpoutSpec that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_state_spout_object = true && this.is_set_state_spout_object();
+    boolean that_present_state_spout_object = true && that.is_set_state_spout_object();
+    if (this_present_state_spout_object || that_present_state_spout_object) {
+      if (!(this_present_state_spout_object && that_present_state_spout_object))
+        return false;
+      if (!this.state_spout_object.equals(that.state_spout_object))
+        return false;
+    }
+
+    boolean this_present_common = true && this.is_set_common();
+    boolean that_present_common = true && that.is_set_common();
+    if (this_present_common || that_present_common) {
+      if (!(this_present_common && that_present_common))
+        return false;
+      if (!this.common.equals(that.common))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_state_spout_object = true && (is_set_state_spout_object());
+    list.add(present_state_spout_object);
+    if (present_state_spout_object)
+      list.add(state_spout_object);
+
+    boolean present_common = true && (is_set_common());
+    list.add(present_common);
+    if (present_common)
+      list.add(common);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(StateSpoutSpec other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_state_spout_object()).compareTo(other.is_set_state_spout_object());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_state_spout_object()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state_spout_object, other.state_spout_object);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common()).compareTo(other.is_set_common());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common, other.common);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StateSpoutSpec(");
+    boolean first = true;
+
+    sb.append("state_spout_object:");
+    if (this.state_spout_object == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.state_spout_object);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("common:");
+    if (this.common == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.common);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_state_spout_object()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'state_spout_object' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_common()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (common != null) {
+      common.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class StateSpoutSpecStandardSchemeFactory implements SchemeFactory {
+    public StateSpoutSpecStandardScheme getScheme() {
+      return new StateSpoutSpecStandardScheme();
+    }
+  }
+
+  private static class StateSpoutSpecStandardScheme extends StandardScheme<StateSpoutSpec> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, StateSpoutSpec struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // STATE_SPOUT_OBJECT
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.state_spout_object = new ComponentObject();
+              struct.state_spout_object.read(iprot);
+              struct.set_state_spout_object_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMMON
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.common = new ComponentCommon();
+              struct.common.read(iprot);
+              struct.set_common_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, StateSpoutSpec struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.state_spout_object != null) {
+        oprot.writeFieldBegin(STATE_SPOUT_OBJECT_FIELD_DESC);
+        struct.state_spout_object.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.common != null) {
+        oprot.writeFieldBegin(COMMON_FIELD_DESC);
+        struct.common.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class StateSpoutSpecTupleSchemeFactory implements SchemeFactory {
+    public StateSpoutSpecTupleScheme getScheme() {
+      return new StateSpoutSpecTupleScheme();
+    }
+  }
+
+  private static class StateSpoutSpecTupleScheme extends TupleScheme<StateSpoutSpec> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, StateSpoutSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.state_spout_object.write(oprot);
+      struct.common.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, StateSpoutSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.state_spout_object = new ComponentObject();
+      struct.state_spout_object.read(iprot);
+      struct.set_state_spout_object_isSet(true);
+      struct.common = new ComponentCommon();
+      struct.common.read(iprot);
+      struct.set_common_isSet(true);
+    }
+  }
+
+}
+


[38/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
new file mode 100644
index 0000000..97b16d8
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor.bolt;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.List;
+import org.apache.storm.Constants;
+import org.apache.storm.ICredentialsListener;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.daemon.metrics.BuiltinMetricsUtil;
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.hooks.info.BoltExecuteInfo;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.task.IBolt;
+import org.apache.storm.task.IOutputCollector;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+public class BoltExecutor extends Executor {
+
+    private static final Logger LOG = LoggerFactory.getLogger(BoltExecutor.class);
+
+    private final Callable<Boolean> executeSampler;
+
+    public BoltExecutor(WorkerState workerData, List<Long> executorId, Map<String, String> credentials) {
+        super(workerData, executorId, credentials);
+        this.executeSampler = ConfigUtils.mkStatsSampler(stormConf);
+    }
+
+    public void init(Map<Integer, Task> idToTask) {
+        while (!stormActive.get()) {
+            Utils.sleep(100);
+        }
+
+        LOG.info("Preparing bolt {}:{}", componentId, idToTask.keySet());
+        for (Map.Entry<Integer, Task> entry : idToTask.entrySet()) {
+            Task taskData = entry.getValue();
+            IBolt boltObject = (IBolt) taskData.getTaskObject();
+            TopologyContext userContext = taskData.getUserContext();
+            taskData.getBuiltInMetrics().registerAll(stormConf, userContext);
+            if (boltObject instanceof ICredentialsListener) {
+                ((ICredentialsListener) boltObject).setCredentials(credentials);
+            }
+            if (Constants.SYSTEM_COMPONENT_ID.equals(componentId)) {
+                Map<String, DisruptorQueue> map = ImmutableMap.of("sendqueue", transferQueue, "receive", receiveQueue,
+                        "transfer", workerData.getTransferQueue());
+                BuiltinMetricsUtil.registerQueueMetrics(map, stormConf, userContext);
+
+                Map cachedNodePortToSocket = (Map) workerData.getCachedNodeToPortSocket().get();
+                BuiltinMetricsUtil.registerIconnectionClientMetrics(cachedNodePortToSocket, stormConf, userContext);
+                BuiltinMetricsUtil.registerIconnectionServerMetric(workerData.getReceiver(), stormConf, userContext);
+            } else {
+                Map<String, DisruptorQueue> map = ImmutableMap.of("sendqueue", transferQueue, "receive", receiveQueue);
+                BuiltinMetricsUtil.registerQueueMetrics(map, stormConf, userContext);
+            }
+
+            IOutputCollector outputCollector = new BoltOutputCollectorImpl(this, taskData, entry.getKey(), rand, hasEventLoggers, isDebug);
+            boltObject.prepare(stormConf, userContext, new OutputCollector(outputCollector));
+        }
+        openOrPrepareWasCalled.set(true);
+        LOG.info("Prepared bolt {}:{}", componentId, idToTask.keySet());
+        setupMetrics();
+    }
+
+    @Override
+    public Callable<Object> call() throws Exception {
+        init(idToTask);
+
+        return new Callable<Object>() {
+            @Override
+            public Object call() throws Exception {
+                receiveQueue.consumeBatchWhenAvailable(BoltExecutor.this);
+                return 0L;
+            }
+        };
+    }
+
+    @Override
+    public void tupleActionFn(int taskId, TupleImpl tuple) throws Exception {
+        String streamId = tuple.getSourceStreamId();
+        if (Constants.CREDENTIALS_CHANGED_STREAM_ID.equals(streamId)) {
+            Object taskObject = idToTask.get(taskId).getTaskObject();
+            if (taskObject instanceof ICredentialsListener) {
+                ((ICredentialsListener) taskObject).setCredentials((Map<String, String>) tuple.getValue(0));
+            }
+        } else if (Constants.METRICS_TICK_STREAM_ID.equals(streamId)) {
+            metricsTick(idToTask.get(taskId), tuple);
+        } else {
+            IBolt boltObject = (IBolt) idToTask.get(taskId).getTaskObject();
+            boolean isSampled = sampler.call();
+            boolean isExecuteSampler = executeSampler.call();
+            Long now = (isSampled || isExecuteSampler) ? System.currentTimeMillis() : null;
+            if (isSampled) {
+                tuple.setProcessSampleStartTime(now);
+            }
+            if (isExecuteSampler) {
+                tuple.setExecuteSampleStartTime(now);
+            }
+            boltObject.execute(tuple);
+
+            Long ms = tuple.getExecuteSampleStartTime();
+            long delta = (ms != null) ? Time.deltaMs(ms) : 0;
+            if (isDebug) {
+                LOG.info("Execute done TUPLE {} TASK: {} DELTA: {}", tuple, taskId, delta);
+            }
+            new BoltExecuteInfo(tuple, taskId, delta).applyOn(idToTask.get(taskId).getUserContext());
+            if (delta != 0) {
+                ((BoltExecutorStats) stats).boltExecuteTuple(tuple.getSourceComponent(), tuple.getSourceStreamId(), delta);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
new file mode 100644
index 0000000..c490c3d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor.bolt;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import org.apache.storm.daemon.Acker;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.hooks.info.BoltAckInfo;
+import org.apache.storm.hooks.info.BoltFailInfo;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.task.IOutputCollector;
+import org.apache.storm.tuple.MessageId;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BoltOutputCollectorImpl implements IOutputCollector {
+
+    private static final Logger LOG = LoggerFactory.getLogger(BoltOutputCollectorImpl.class);
+
+    private final BoltExecutor executor;
+    private final Task taskData;
+    private final int taskId;
+    private final Random random;
+    private final boolean isEventLoggers;
+    private final boolean isDebug;
+
+    public BoltOutputCollectorImpl(BoltExecutor executor, Task taskData, int taskId, Random random,
+                                   boolean isEventLoggers, boolean isDebug) {
+        this.executor = executor;
+        this.taskData = taskData;
+        this.taskId = taskId;
+        this.random = random;
+        this.isEventLoggers = isEventLoggers;
+        this.isDebug = isDebug;
+    }
+
+    public List<Integer> emit(String streamId, Collection<Tuple> anchors, List<Object> tuple) {
+        return boltEmit(streamId, anchors, tuple, null);
+    }
+
+    @Override
+    public void emitDirect(int taskId, String streamId, Collection<Tuple> anchors, List<Object> tuple) {
+        boltEmit(streamId, anchors, tuple, taskId);
+    }
+
+    private List<Integer> boltEmit(String streamId, Collection<Tuple> anchors, List<Object> values, Integer targetTaskId) {
+        List<Integer> outTasks;
+        if (targetTaskId != null) {
+            outTasks = taskData.getOutgoingTasks(targetTaskId, streamId, values);
+        } else {
+            outTasks = taskData.getOutgoingTasks(streamId, values);
+        }
+
+        for (Integer t : outTasks) {
+            Map<Long, Long> anchorsToIds = new HashMap<>();
+            if (anchors != null) {
+                for (Tuple a : anchors) {
+                    Set<Long> rootIds = a.getMessageId().getAnchorsToIds().keySet();
+                    if (rootIds.size() > 0) {
+                        long edgeId = MessageId.generateId(random);
+                        ((TupleImpl) a).updateAckVal(edgeId);
+                        for (Long root_id : rootIds) {
+                            putXor(anchorsToIds, root_id, edgeId);
+                        }
+                    }
+                }
+            }
+            MessageId msgId = MessageId.makeId(anchorsToIds);
+            TupleImpl tupleExt = new TupleImpl(executor.getWorkerTopologyContext(), values, taskId, streamId, msgId);
+            executor.getExecutorTransfer().transfer(t, tupleExt);
+        }
+        if (isEventLoggers) {
+            executor.sendToEventLogger(executor, taskData, values, executor.getComponentId(), null, random);
+        }
+        return outTasks;
+    }
+
+    @Override
+    public void ack(Tuple input) {
+        long ackValue = ((TupleImpl) input).getAckVal();
+        Map<Long, Long> anchorsToIds = input.getMessageId().getAnchorsToIds();
+        for (Map.Entry<Long, Long> entry : anchorsToIds.entrySet()) {
+            executor.sendUnanchored(taskData, Acker.ACKER_ACK_STREAM_ID,
+                    new Values(entry.getKey(), Utils.bitXor(entry.getValue(), ackValue)),
+                    executor.getExecutorTransfer());
+        }
+        long delta = tupleTimeDelta((TupleImpl) input);
+        if (isDebug) {
+            LOG.info("BOLT ack TASK: {} TIME: {} TUPLE: {}", taskId, delta, input);
+        }
+        BoltAckInfo boltAckInfo = new BoltAckInfo(input, taskId, delta);
+        boltAckInfo.applyOn(taskData.getUserContext());
+        if (delta != 0) {
+            ((BoltExecutorStats) executor.getStats()).boltAckedTuple(
+                    input.getSourceComponent(), input.getSourceStreamId(), delta);
+        }
+    }
+
+    @Override
+    public void fail(Tuple input) {
+        Set<Long> roots = input.getMessageId().getAnchors();
+        for (Long root : roots) {
+            executor.sendUnanchored(taskData, Acker.ACKER_FAIL_STREAM_ID,
+                    new Values(root), executor.getExecutorTransfer());
+        }
+        long delta = tupleTimeDelta((TupleImpl) input);
+        if (isDebug) {
+            LOG.info("BOLT fail TASK: {} TIME: {} TUPLE: {}", taskId, delta, input);
+        }
+        BoltFailInfo boltFailInfo = new BoltFailInfo(input, taskId, delta);
+        boltFailInfo.applyOn(taskData.getUserContext());
+        if (delta != 0) {
+            ((BoltExecutorStats) executor.getStats()).boltFailedTuple(
+                    input.getSourceComponent(), input.getSourceStreamId(), delta);
+        }
+    }
+
+    @Override
+    public void resetTimeout(Tuple input) {
+        Set<Long> roots = input.getMessageId().getAnchors();
+        for (Long root : roots) {
+            executor.sendUnanchored(taskData, Acker.ACKER_RESET_TIMEOUT_STREAM_ID,
+                    new Values(root), executor.getExecutorTransfer());
+        }
+    }
+
+    @Override
+    public void reportError(Throwable error) {
+        executor.getReportError().report(error);
+    }
+
+    private long tupleTimeDelta(TupleImpl tuple) {
+        Long ms = tuple.getProcessSampleStartTime();
+        if (ms != null)
+            return Time.deltaMs(ms);
+        return 0;
+    }
+
+    private void putXor(Map<Long, Long> pending, Long key, Long id) {
+        Long curr = pending.get(key);
+        if (curr == null) {
+            curr = 0l;
+        }
+        pending.put(key, Utils.bitXor(curr, id));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/error/IReportError.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/error/IReportError.java b/storm-client/src/jvm/org/apache/storm/executor/error/IReportError.java
new file mode 100644
index 0000000..73451f8
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/error/IReportError.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor.error;
+
+public interface IReportError {
+    void report(Throwable error);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/error/ReportError.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/error/ReportError.java b/storm-client/src/jvm/org/apache/storm/executor/error/ReportError.java
new file mode 100644
index 0000000..3272b85
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/error/ReportError.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor.error;
+
+import org.apache.storm.Config;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class ReportError implements IReportError {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ReportError.class);
+
+    private final Map stormConf;
+    private final IStormClusterState stormClusterState;
+    private final String stormId;
+    private final String componentId;
+    private final WorkerTopologyContext workerTopologyContext;
+
+    private int maxPerInterval;
+    private int errorIntervalSecs;
+    private AtomicInteger intervalStartTime;
+    private AtomicInteger intervalErrors;
+
+    public ReportError(Map stormConf, IStormClusterState stormClusterState, String stormId, String componentId, WorkerTopologyContext workerTopologyContext) {
+        this.stormConf = stormConf;
+        this.stormClusterState = stormClusterState;
+        this.stormId = stormId;
+        this.componentId = componentId;
+        this.workerTopologyContext = workerTopologyContext;
+        this.errorIntervalSecs = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS));
+        this.maxPerInterval = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL));
+        this.intervalStartTime = new AtomicInteger(Time.currentTimeSecs());
+        this.intervalErrors = new AtomicInteger(0);
+    }
+
+    @Override
+    public void report(Throwable error) {
+        LOG.error("Error", error);
+        if (Time.deltaSecs(intervalStartTime.get()) > errorIntervalSecs) {
+            intervalErrors.set(0);
+            intervalStartTime.set(Time.currentTimeSecs());
+        }
+        if (intervalErrors.incrementAndGet() <= maxPerInterval) {
+            try {
+                stormClusterState.reportError(stormId, componentId, Utils.hostname(),
+                        workerTopologyContext.getThisWorkerPort().longValue(), error);
+            } catch (UnknownHostException e) {
+                throw Utils.wrapInRuntime(e);
+            }
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/error/ReportErrorAndDie.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/error/ReportErrorAndDie.java b/storm-client/src/jvm/org/apache/storm/executor/error/ReportErrorAndDie.java
new file mode 100644
index 0000000..b2e1f34
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/error/ReportErrorAndDie.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor.error;
+
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReportErrorAndDie implements Thread.UncaughtExceptionHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(ReportErrorAndDie.class);
+    private final IReportError reportError;
+    private final Runnable suicideFn;
+
+    public ReportErrorAndDie(IReportError reportError, Runnable suicideFn) {
+        this.reportError = reportError;
+        this.suicideFn = suicideFn;
+    }
+
+    @Override
+    public void uncaughtException(Thread t, Throwable e) {
+        try {
+            reportError.report(e);
+        } catch (Exception ex) {
+            LOG.error("Error while reporting error to cluster, proceeding with shutdown", ex);
+        }
+        if (Utils.exceptionCauseIsInstanceOf(InterruptedException.class, e)
+                || (Utils.exceptionCauseIsInstanceOf(java.io.InterruptedIOException.class, e)
+                    && !Utils.exceptionCauseIsInstanceOf(java.net.SocketTimeoutException.class, e))) {
+            LOG.info("Got interrupted exception shutting thread down...");
+        } else {
+            suicideFn.run();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
new file mode 100644
index 0000000..9e7622c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
@@ -0,0 +1,259 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor.spout;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.concurrent.Callable;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.ICredentialsListener;
+import org.apache.storm.daemon.Acker;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.daemon.metrics.BuiltinMetricsUtil;
+import org.apache.storm.daemon.metrics.SpoutThrottlingMetrics;
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.TupleInfo;
+import org.apache.storm.hooks.info.SpoutAckInfo;
+import org.apache.storm.hooks.info.SpoutFailInfo;
+import org.apache.storm.spout.ISpout;
+import org.apache.storm.spout.ISpoutWaitStrategy;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.MutableLong;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.ReflectionUtils;
+import org.apache.storm.utils.RotatingMap;
+import org.apache.storm.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class SpoutExecutor extends Executor {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SpoutExecutor.class);
+
+    private final ISpoutWaitStrategy spoutWaitStrategy;
+    private Integer maxSpoutPending;
+    private final AtomicBoolean lastActive;
+    private List<ISpout> spouts;
+    private List<SpoutOutputCollector> outputCollectors;
+    private final MutableLong emittedCount;
+    private final MutableLong emptyEmitStreak;
+    private final SpoutThrottlingMetrics spoutThrottlingMetrics;
+    private final boolean hasAckers;
+    private RotatingMap<Long, TupleInfo> pending;
+    private final boolean backPressureEnabled;
+
+    public SpoutExecutor(final WorkerState workerData, final List<Long> executorId, Map<String, String> credentials) {
+        super(workerData, executorId, credentials);
+        this.spoutWaitStrategy = ReflectionUtils.newInstance((String) stormConf.get(Config.TOPOLOGY_SPOUT_WAIT_STRATEGY));
+        this.spoutWaitStrategy.prepare(stormConf);
+
+        this.backPressureEnabled = ObjectReader.getBoolean(stormConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE), false);
+
+        this.lastActive = new AtomicBoolean(false);
+        this.hasAckers = StormCommon.hasAckers(stormConf);
+        this.emittedCount = new MutableLong(0);
+        this.emptyEmitStreak = new MutableLong(0);
+        this.spoutThrottlingMetrics = new SpoutThrottlingMetrics();
+    }
+
+    public void init(final Map<Integer, Task> idToTask) {
+        while (!stormActive.get()) {
+            Utils.sleep(100);
+        }
+
+        LOG.info("Opening spout {}:{}", componentId, idToTask.keySet());
+        this.idToTask = idToTask;
+        this.maxSpoutPending = ObjectReader.getInt(stormConf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING), 0) * idToTask.size();
+        this.spouts = new ArrayList<>();
+        for (Task task : idToTask.values()) {
+            this.spouts.add((ISpout) task.getTaskObject());
+        }
+        this.pending = new RotatingMap<>(2, new RotatingMap.ExpiredCallback<Long, TupleInfo>() {
+            @Override
+            public void expire(Long key, TupleInfo tupleInfo) {
+                Long timeDelta = null;
+                if (tupleInfo.getTimestamp() != 0) {
+                    timeDelta = Time.deltaMs(tupleInfo.getTimestamp());
+                }
+                failSpoutMsg(SpoutExecutor.this, idToTask.get(tupleInfo.getTaskId()), timeDelta, tupleInfo, "TIMEOUT");
+            }
+        });
+
+        this.spoutThrottlingMetrics.registerAll(stormConf, idToTask.values().iterator().next().getUserContext());
+        this.outputCollectors = new ArrayList<>();
+        for (Map.Entry<Integer, Task> entry : idToTask.entrySet()) {
+            Task taskData = entry.getValue();
+            ISpout spoutObject = (ISpout) taskData.getTaskObject();
+            SpoutOutputCollectorImpl spoutOutputCollector = new SpoutOutputCollectorImpl(
+                    spoutObject, this, taskData, entry.getKey(), emittedCount,
+                    hasAckers, rand, hasEventLoggers, isDebug, pending);
+            SpoutOutputCollector outputCollector = new SpoutOutputCollector(spoutOutputCollector);
+            this.outputCollectors.add(outputCollector);
+
+            taskData.getBuiltInMetrics().registerAll(stormConf, taskData.getUserContext());
+            Map<String, DisruptorQueue> map = ImmutableMap.of("sendqueue", transferQueue, "receive", receiveQueue);
+            BuiltinMetricsUtil.registerQueueMetrics(map, stormConf, taskData.getUserContext());
+
+            if (spoutObject instanceof ICredentialsListener) {
+                ((ICredentialsListener) spoutObject).setCredentials(credentials);
+            }
+            spoutObject.open(stormConf, taskData.getUserContext(), outputCollector);
+        }
+        openOrPrepareWasCalled.set(true);
+        LOG.info("Opened spout {}:{}", componentId, idToTask.keySet());
+        setupMetrics();
+    }
+
+    @Override
+    public Callable<Object> call() throws Exception {
+        init(idToTask);
+
+        return new Callable<Object>() {
+            @Override
+            public Object call() throws Exception {
+                receiveQueue.consumeBatch(SpoutExecutor.this);
+
+                long currCount = emittedCount.get();
+                boolean throttleOn = backPressureEnabled && SpoutExecutor.this.throttleOn.get();
+                boolean reachedMaxSpoutPending = (maxSpoutPending != 0) && (pending.size() >= maxSpoutPending);
+                boolean isActive = stormActive.get();
+                if (isActive) {
+                    if (!lastActive.get()) {
+                        lastActive.set(true);
+                        LOG.info("Activating spout {}:{}", componentId, idToTask.keySet());
+                        for (ISpout spout : spouts) {
+                            spout.activate();
+                        }
+                    }
+                    if (!transferQueue.isFull() && !throttleOn && !reachedMaxSpoutPending) {
+                        for (ISpout spout : spouts) {
+                            spout.nextTuple();
+                        }
+                    }
+                } else {
+                    if (lastActive.get()) {
+                        lastActive.set(false);
+                        LOG.info("Deactivating spout {}:{}", componentId, idToTask.keySet());
+                        for (ISpout spout : spouts) {
+                            spout.deactivate();
+                        }
+                    }
+                    Time.sleep(100);
+                    spoutThrottlingMetrics.skippedInactive(stats);
+                }
+                if (currCount == emittedCount.get() && isActive) {
+                    emptyEmitStreak.increment();
+                    spoutWaitStrategy.emptyEmit(emptyEmitStreak.get());
+                    if (throttleOn) {
+                        spoutThrottlingMetrics.skippedThrottle(stats);
+                    } else if (reachedMaxSpoutPending) {
+                        spoutThrottlingMetrics.skippedMaxSpout(stats);
+                    }
+                } else {
+                    emptyEmitStreak.set(0);
+                }
+                return 0L;
+            }
+        };
+    }
+
+    @Override
+    public void tupleActionFn(int taskId, TupleImpl tuple) throws Exception {
+        String streamId = tuple.getSourceStreamId();
+        if (streamId.equals(Constants.SYSTEM_TICK_STREAM_ID)) {
+            pending.rotate();
+        } else if (streamId.equals(Constants.METRICS_TICK_STREAM_ID)) {
+            metricsTick(idToTask.get(taskId), tuple);
+        } else if (streamId.equals(Constants.CREDENTIALS_CHANGED_STREAM_ID)) {
+            Object spoutObj = idToTask.get(taskId).getTaskObject();
+            if (spoutObj instanceof ICredentialsListener) {
+                ((ICredentialsListener) spoutObj).setCredentials((Map<String, String>) tuple.getValue(0));
+            }
+        } else if (streamId.equals(Acker.ACKER_RESET_TIMEOUT_STREAM_ID)) {
+            Long id = (Long) tuple.getValue(0);
+            TupleInfo pendingForId = pending.get(id);
+            if (pendingForId != null) {
+                pending.put(id, pendingForId);
+            }
+        } else {
+            Long id = (Long) tuple.getValue(0);
+            Long timeDeltaMs = (Long) tuple.getValue(1);
+            TupleInfo tupleInfo = (TupleInfo) pending.remove(id);
+            if (tupleInfo != null && tupleInfo.getMessageId() != null) {
+                if (taskId != tupleInfo.getTaskId()) {
+                    throw new RuntimeException("Fatal error, mismatched task ids: " + taskId + " " + tupleInfo.getTaskId());
+                }
+                long startTimeMs = tupleInfo.getTimestamp();
+                Long timeDelta = null;
+                if (startTimeMs != 0) {
+                    timeDelta = timeDeltaMs;
+                }
+                if (streamId.equals(Acker.ACKER_ACK_STREAM_ID)) {
+                    ackSpoutMsg(this, idToTask.get(taskId), timeDelta, tupleInfo);
+                } else if (streamId.equals(Acker.ACKER_FAIL_STREAM_ID)) {
+                    failSpoutMsg(this, idToTask.get(taskId), timeDelta, tupleInfo, "FAIL-STREAM");
+                }
+            }
+        }
+    }
+
+    public void ackSpoutMsg(Executor executor, Task taskData, Long timeDelta, TupleInfo tupleInfo) {
+        try {
+            ISpout spout = (ISpout) taskData.getTaskObject();
+            int taskId = taskData.getTaskId();
+            if (executor.getIsDebug()) {
+                LOG.info("SPOUT Acking message {} {}", tupleInfo.getId(), tupleInfo.getMessageId());
+            }
+            spout.ack(tupleInfo.getMessageId());
+            new SpoutAckInfo(tupleInfo.getMessageId(), taskId, timeDelta).applyOn(taskData.getUserContext());
+            if (timeDelta != null) {
+                ((SpoutExecutorStats) executor.getStats()).spoutAckedTuple(tupleInfo.getStream(), timeDelta);
+            }
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    public void failSpoutMsg(Executor executor, Task taskData, Long timeDelta, TupleInfo tupleInfo, String reason) {
+        try {
+            ISpout spout = (ISpout) taskData.getTaskObject();
+            int taskId = taskData.getTaskId();
+            if (executor.getIsDebug()) {
+                LOG.info("SPOUT Failing {} : {} REASON: {}", tupleInfo.getId(), tupleInfo, reason);
+            }
+            spout.fail(tupleInfo.getMessageId());
+            new SpoutFailInfo(tupleInfo.getMessageId(), taskId, timeDelta).applyOn(taskData.getUserContext());
+            if (timeDelta != null) {
+                ((SpoutExecutorStats) executor.getStats()).spoutFailedTuple(tupleInfo.getStream(), timeDelta);
+            }
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java
new file mode 100644
index 0000000..f81b2c2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.executor.spout;
+
+import org.apache.storm.daemon.Acker;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.TupleInfo;
+import org.apache.storm.spout.ISpout;
+import org.apache.storm.spout.ISpoutOutputCollector;
+import org.apache.storm.tuple.MessageId;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.MutableLong;
+import org.apache.storm.utils.RotatingMap;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+public class SpoutOutputCollectorImpl implements ISpoutOutputCollector {
+
+    private final SpoutExecutor executor;
+    private final Task taskData;
+    private final int taskId;
+    private final MutableLong emittedCount;
+    private final boolean hasAckers;
+    private final Random random;
+    private final Boolean isEventLoggers;
+    private final Boolean isDebug;
+    private final RotatingMap<Long, TupleInfo> pending;
+
+    @SuppressWarnings("unused")
+    public SpoutOutputCollectorImpl(ISpout spout, SpoutExecutor executor, Task taskData, int taskId,
+                                    MutableLong emittedCount, boolean hasAckers, Random random,
+                                    Boolean isEventLoggers, Boolean isDebug, RotatingMap<Long, TupleInfo> pending) {
+        this.executor = executor;
+        this.taskData = taskData;
+        this.taskId = taskId;
+        this.emittedCount = emittedCount;
+        this.hasAckers = hasAckers;
+        this.random = random;
+        this.isEventLoggers = isEventLoggers;
+        this.isDebug = isDebug;
+        this.pending = pending;
+    }
+
+    @Override
+    public List<Integer> emit(String streamId, List<Object> tuple, Object messageId) {
+        return sendSpoutMsg(streamId, tuple, messageId, null);
+    }
+
+    @Override
+    public void emitDirect(int taskId, String streamId, List<Object> tuple, Object messageId) {
+        sendSpoutMsg(streamId, tuple, messageId, taskId);
+    }
+
+    @Override
+    public long getPendingCount() {
+        return pending.size();
+    }
+
+    @Override
+    public void reportError(Throwable error) {
+        executor.getReportError().report(error);
+    }
+
+    private List<Integer> sendSpoutMsg(String stream, List<Object> values, Object messageId, Integer outTaskId) {
+        emittedCount.increment();
+
+        List<Integer> outTasks;
+        if (outTaskId != null) {
+            outTasks = taskData.getOutgoingTasks(outTaskId, stream, values);
+        } else {
+            outTasks = taskData.getOutgoingTasks(stream, values);
+        }
+
+        List<Long> ackSeq = new ArrayList<>();
+        boolean needAck = (messageId != null) && hasAckers;
+
+        long rootId = MessageId.generateId(random);
+        for (Integer t : outTasks) {
+            MessageId msgId;
+            if (needAck) {
+                long as = MessageId.generateId(random);
+                msgId = MessageId.makeRootId(rootId, as);
+                ackSeq.add(as);
+            } else {
+                msgId = MessageId.makeUnanchored();
+            }
+
+            TupleImpl tuple = new TupleImpl(executor.getWorkerTopologyContext(), values, this.taskId, stream, msgId);
+            executor.getExecutorTransfer().transfer(t, tuple);
+        }
+        if (isEventLoggers) {
+            executor.sendToEventLogger(executor, taskData, values, executor.getComponentId(), messageId, random);
+        }
+
+        boolean sample = false;
+        try {
+            sample = executor.getSampler().call();
+        } catch (Exception ignored) {
+        }
+        if (needAck) {
+            TupleInfo info = new TupleInfo();
+            info.setTaskId(this.taskId);
+            info.setStream(stream);
+            info.setMessageId(messageId);
+            if (isDebug) {
+                info.setValues(values);
+            }
+            if (sample) {
+                info.setTimestamp(System.currentTimeMillis());
+            }
+
+            pending.put(rootId, info);
+            List<Object> ackInitTuple = new Values(rootId, Utils.bitXorVals(ackSeq), this.taskId);
+            executor.sendUnanchored(taskData, Acker.ACKER_INIT_STREAM_ID, ackInitTuple, executor.getExecutorTransfer());
+        } else if (messageId != null) {
+            TupleInfo info = new TupleInfo();
+            info.setStream(stream);
+            info.setValues(values);
+            info.setMessageId(messageId);
+            info.setTimestamp(0);
+            Long timeDelta = sample ? 0L : null;
+            info.setId("0:");
+            executor.ackSpoutMsg(executor, taskData, timeDelta, info);
+        }
+
+        return outTasks;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/AccessControl.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/AccessControl.java b/storm-client/src/jvm/org/apache/storm/generated/AccessControl.java
new file mode 100644
index 0000000..c1c072a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/AccessControl.java
@@ -0,0 +1,627 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class AccessControl implements org.apache.thrift.TBase<AccessControl, AccessControl._Fields>, java.io.Serializable, Cloneable, Comparable<AccessControl> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AccessControl");
+
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ACCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("access", org.apache.thrift.protocol.TType.I32, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AccessControlStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AccessControlTupleSchemeFactory());
+  }
+
+  private AccessControlType type; // required
+  private String name; // optional
+  private int access; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see AccessControlType
+     */
+    TYPE((short)1, "type"),
+    NAME((short)2, "name"),
+    ACCESS((short)3, "access");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TYPE
+          return TYPE;
+        case 2: // NAME
+          return NAME;
+        case 3: // ACCESS
+          return ACCESS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ACCESS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NAME};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, AccessControlType.class)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ACCESS, new org.apache.thrift.meta_data.FieldMetaData("access", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AccessControl.class, metaDataMap);
+  }
+
+  public AccessControl() {
+  }
+
+  public AccessControl(
+    AccessControlType type,
+    int access)
+  {
+    this();
+    this.type = type;
+    this.access = access;
+    set_access_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AccessControl(AccessControl other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_type()) {
+      this.type = other.type;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.access = other.access;
+  }
+
+  public AccessControl deepCopy() {
+    return new AccessControl(this);
+  }
+
+  @Override
+  public void clear() {
+    this.type = null;
+    this.name = null;
+    set_access_isSet(false);
+    this.access = 0;
+  }
+
+  /**
+   * 
+   * @see AccessControlType
+   */
+  public AccessControlType get_type() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see AccessControlType
+   */
+  public void set_type(AccessControlType type) {
+    this.type = type;
+  }
+
+  public void unset_type() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_type() {
+    return this.type != null;
+  }
+
+  public void set_type_isSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_access() {
+    return this.access;
+  }
+
+  public void set_access(int access) {
+    this.access = access;
+    set_access_isSet(true);
+  }
+
+  public void unset_access() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ACCESS_ISSET_ID);
+  }
+
+  /** Returns true if field access is set (has been assigned a value) and false otherwise */
+  public boolean is_set_access() {
+    return EncodingUtils.testBit(__isset_bitfield, __ACCESS_ISSET_ID);
+  }
+
+  public void set_access_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ACCESS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TYPE:
+      if (value == null) {
+        unset_type();
+      } else {
+        set_type((AccessControlType)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case ACCESS:
+      if (value == null) {
+        unset_access();
+      } else {
+        set_access((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TYPE:
+      return get_type();
+
+    case NAME:
+      return get_name();
+
+    case ACCESS:
+      return get_access();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TYPE:
+      return is_set_type();
+    case NAME:
+      return is_set_name();
+    case ACCESS:
+      return is_set_access();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AccessControl)
+      return this.equals((AccessControl)that);
+    return false;
+  }
+
+  public boolean equals(AccessControl that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_type = true && this.is_set_type();
+    boolean that_present_type = true && that.is_set_type();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_access = true;
+    boolean that_present_access = true;
+    if (this_present_access || that_present_access) {
+      if (!(this_present_access && that_present_access))
+        return false;
+      if (this.access != that.access)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_type = true && (is_set_type());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_access = true;
+    list.add(present_access);
+    if (present_access)
+      list.add(access);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AccessControl other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_access()).compareTo(other.is_set_access());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_access()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.access, other.access);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AccessControl(");
+    boolean first = true;
+
+    sb.append("type:");
+    if (this.type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.type);
+    }
+    first = false;
+    if (is_set_name()) {
+      if (!first) sb.append(", ");
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+    }
+    if (!first) sb.append(", ");
+    sb.append("access:");
+    sb.append(this.access);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_type()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_access()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'access' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class AccessControlStandardSchemeFactory implements SchemeFactory {
+    public AccessControlStandardScheme getScheme() {
+      return new AccessControlStandardScheme();
+    }
+  }
+
+  private static class AccessControlStandardScheme extends StandardScheme<AccessControl> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AccessControl struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = org.apache.storm.generated.AccessControlType.findByValue(iprot.readI32());
+              struct.set_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ACCESS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.access = iprot.readI32();
+              struct.set_access_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, AccessControl struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.type != null) {
+        oprot.writeFieldBegin(TYPE_FIELD_DESC);
+        oprot.writeI32(struct.type.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.name != null) {
+        if (struct.is_set_name()) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldBegin(ACCESS_FIELD_DESC);
+      oprot.writeI32(struct.access);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AccessControlTupleSchemeFactory implements SchemeFactory {
+    public AccessControlTupleScheme getScheme() {
+      return new AccessControlTupleScheme();
+    }
+  }
+
+  private static class AccessControlTupleScheme extends TupleScheme<AccessControl> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AccessControl struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.type.getValue());
+      oprot.writeI32(struct.access);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_name()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_name()) {
+        oprot.writeString(struct.name);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, AccessControl struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.type = org.apache.storm.generated.AccessControlType.findByValue(iprot.readI32());
+      struct.set_type_isSet(true);
+      struct.access = iprot.readI32();
+      struct.set_access_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.name = iprot.readString();
+        struct.set_name_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/AccessControlType.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/AccessControlType.java b/storm-client/src/jvm/org/apache/storm/generated/AccessControlType.java
new file mode 100644
index 0000000..cb75b9b
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/AccessControlType.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum AccessControlType implements org.apache.thrift.TEnum {
+  OTHER(1),
+  USER(2);
+
+  private final int value;
+
+  private AccessControlType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static AccessControlType findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return OTHER;
+      case 2:
+        return USER;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/AlreadyAliveException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/AlreadyAliveException.java b/storm-client/src/jvm/org/apache/storm/generated/AlreadyAliveException.java
new file mode 100644
index 0000000..4e635fb
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/AlreadyAliveException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class AlreadyAliveException extends TException implements org.apache.thrift.TBase<AlreadyAliveException, AlreadyAliveException._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyAliveException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyAliveException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AlreadyAliveExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AlreadyAliveExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AlreadyAliveException.class, metaDataMap);
+  }
+
+  public AlreadyAliveException() {
+  }
+
+  public AlreadyAliveException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AlreadyAliveException(AlreadyAliveException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public AlreadyAliveException deepCopy() {
+    return new AlreadyAliveException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AlreadyAliveException)
+      return this.equals((AlreadyAliveException)that);
+    return false;
+  }
+
+  public boolean equals(AlreadyAliveException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AlreadyAliveException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AlreadyAliveException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class AlreadyAliveExceptionStandardSchemeFactory implements SchemeFactory {
+    public AlreadyAliveExceptionStandardScheme getScheme() {
+      return new AlreadyAliveExceptionStandardScheme();
+    }
+  }
+
+  private static class AlreadyAliveExceptionStandardScheme extends StandardScheme<AlreadyAliveException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AlreadyAliveException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, AlreadyAliveException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AlreadyAliveExceptionTupleSchemeFactory implements SchemeFactory {
+    public AlreadyAliveExceptionTupleScheme getScheme() {
+      return new AlreadyAliveExceptionTupleScheme();
+    }
+  }
+
+  private static class AlreadyAliveExceptionTupleScheme extends TupleScheme<AlreadyAliveException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AlreadyAliveException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, AlreadyAliveException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+


[37/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/Assignment.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/Assignment.java b/storm-client/src/jvm/org/apache/storm/generated/Assignment.java
new file mode 100644
index 0000000..4c973d5
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/Assignment.java
@@ -0,0 +1,1159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable, Comparable<Assignment> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment");
+
+  private static final org.apache.thrift.protocol.TField MASTER_CODE_DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("master_code_dir", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField NODE_HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("node_host", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_NODE_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_node_port", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_START_TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_start_time_secs", org.apache.thrift.protocol.TType.MAP, (short)4);
+  private static final org.apache.thrift.protocol.TField WORKER_RESOURCES_FIELD_DESC = new org.apache.thrift.protocol.TField("worker_resources", org.apache.thrift.protocol.TType.MAP, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AssignmentStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AssignmentTupleSchemeFactory());
+  }
+
+  private String master_code_dir; // required
+  private Map<String,String> node_host; // optional
+  private Map<List<Long>,NodeInfo> executor_node_port; // optional
+  private Map<List<Long>,Long> executor_start_time_secs; // optional
+  private Map<NodeInfo,WorkerResources> worker_resources; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MASTER_CODE_DIR((short)1, "master_code_dir"),
+    NODE_HOST((short)2, "node_host"),
+    EXECUTOR_NODE_PORT((short)3, "executor_node_port"),
+    EXECUTOR_START_TIME_SECS((short)4, "executor_start_time_secs"),
+    WORKER_RESOURCES((short)5, "worker_resources");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MASTER_CODE_DIR
+          return MASTER_CODE_DIR;
+        case 2: // NODE_HOST
+          return NODE_HOST;
+        case 3: // EXECUTOR_NODE_PORT
+          return EXECUTOR_NODE_PORT;
+        case 4: // EXECUTOR_START_TIME_SECS
+          return EXECUTOR_START_TIME_SECS;
+        case 5: // WORKER_RESOURCES
+          return WORKER_RESOURCES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.NODE_HOST,_Fields.EXECUTOR_NODE_PORT,_Fields.EXECUTOR_START_TIME_SECS,_Fields.WORKER_RESOURCES};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MASTER_CODE_DIR, new org.apache.thrift.meta_data.FieldMetaData("master_code_dir", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NODE_HOST, new org.apache.thrift.meta_data.FieldMetaData("node_host", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.EXECUTOR_NODE_PORT, new org.apache.thrift.meta_data.FieldMetaData("executor_node_port", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NodeInfo.class))));
+    tmpMap.put(_Fields.EXECUTOR_START_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("executor_start_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WORKER_RESOURCES, new org.apache.thrift.meta_data.FieldMetaData("worker_resources", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NodeInfo.class), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerResources.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Assignment.class, metaDataMap);
+  }
+
+  public Assignment() {
+    this.node_host = new HashMap<String,String>();
+
+    this.executor_node_port = new HashMap<List<Long>,NodeInfo>();
+
+    this.executor_start_time_secs = new HashMap<List<Long>,Long>();
+
+    this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+
+  }
+
+  public Assignment(
+    String master_code_dir)
+  {
+    this();
+    this.master_code_dir = master_code_dir;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Assignment(Assignment other) {
+    if (other.is_set_master_code_dir()) {
+      this.master_code_dir = other.master_code_dir;
+    }
+    if (other.is_set_node_host()) {
+      Map<String,String> __this__node_host = new HashMap<String,String>(other.node_host);
+      this.node_host = __this__node_host;
+    }
+    if (other.is_set_executor_node_port()) {
+      Map<List<Long>,NodeInfo> __this__executor_node_port = new HashMap<List<Long>,NodeInfo>(other.executor_node_port.size());
+      for (Map.Entry<List<Long>, NodeInfo> other_element : other.executor_node_port.entrySet()) {
+
+        List<Long> other_element_key = other_element.getKey();
+        NodeInfo other_element_value = other_element.getValue();
+
+        List<Long> __this__executor_node_port_copy_key = new ArrayList<Long>(other_element_key);
+
+        NodeInfo __this__executor_node_port_copy_value = new NodeInfo(other_element_value);
+
+        __this__executor_node_port.put(__this__executor_node_port_copy_key, __this__executor_node_port_copy_value);
+      }
+      this.executor_node_port = __this__executor_node_port;
+    }
+    if (other.is_set_executor_start_time_secs()) {
+      Map<List<Long>,Long> __this__executor_start_time_secs = new HashMap<List<Long>,Long>(other.executor_start_time_secs.size());
+      for (Map.Entry<List<Long>, Long> other_element : other.executor_start_time_secs.entrySet()) {
+
+        List<Long> other_element_key = other_element.getKey();
+        Long other_element_value = other_element.getValue();
+
+        List<Long> __this__executor_start_time_secs_copy_key = new ArrayList<Long>(other_element_key);
+
+        Long __this__executor_start_time_secs_copy_value = other_element_value;
+
+        __this__executor_start_time_secs.put(__this__executor_start_time_secs_copy_key, __this__executor_start_time_secs_copy_value);
+      }
+      this.executor_start_time_secs = __this__executor_start_time_secs;
+    }
+    if (other.is_set_worker_resources()) {
+      Map<NodeInfo,WorkerResources> __this__worker_resources = new HashMap<NodeInfo,WorkerResources>(other.worker_resources.size());
+      for (Map.Entry<NodeInfo, WorkerResources> other_element : other.worker_resources.entrySet()) {
+
+        NodeInfo other_element_key = other_element.getKey();
+        WorkerResources other_element_value = other_element.getValue();
+
+        NodeInfo __this__worker_resources_copy_key = new NodeInfo(other_element_key);
+
+        WorkerResources __this__worker_resources_copy_value = new WorkerResources(other_element_value);
+
+        __this__worker_resources.put(__this__worker_resources_copy_key, __this__worker_resources_copy_value);
+      }
+      this.worker_resources = __this__worker_resources;
+    }
+  }
+
+  public Assignment deepCopy() {
+    return new Assignment(this);
+  }
+
+  @Override
+  public void clear() {
+    this.master_code_dir = null;
+    this.node_host = new HashMap<String,String>();
+
+    this.executor_node_port = new HashMap<List<Long>,NodeInfo>();
+
+    this.executor_start_time_secs = new HashMap<List<Long>,Long>();
+
+    this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+
+  }
+
+  public String get_master_code_dir() {
+    return this.master_code_dir;
+  }
+
+  public void set_master_code_dir(String master_code_dir) {
+    this.master_code_dir = master_code_dir;
+  }
+
+  public void unset_master_code_dir() {
+    this.master_code_dir = null;
+  }
+
+  /** Returns true if field master_code_dir is set (has been assigned a value) and false otherwise */
+  public boolean is_set_master_code_dir() {
+    return this.master_code_dir != null;
+  }
+
+  public void set_master_code_dir_isSet(boolean value) {
+    if (!value) {
+      this.master_code_dir = null;
+    }
+  }
+
+  public int get_node_host_size() {
+    return (this.node_host == null) ? 0 : this.node_host.size();
+  }
+
+  public void put_to_node_host(String key, String val) {
+    if (this.node_host == null) {
+      this.node_host = new HashMap<String,String>();
+    }
+    this.node_host.put(key, val);
+  }
+
+  public Map<String,String> get_node_host() {
+    return this.node_host;
+  }
+
+  public void set_node_host(Map<String,String> node_host) {
+    this.node_host = node_host;
+  }
+
+  public void unset_node_host() {
+    this.node_host = null;
+  }
+
+  /** Returns true if field node_host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_node_host() {
+    return this.node_host != null;
+  }
+
+  public void set_node_host_isSet(boolean value) {
+    if (!value) {
+      this.node_host = null;
+    }
+  }
+
+  public int get_executor_node_port_size() {
+    return (this.executor_node_port == null) ? 0 : this.executor_node_port.size();
+  }
+
+  public void put_to_executor_node_port(List<Long> key, NodeInfo val) {
+    if (this.executor_node_port == null) {
+      this.executor_node_port = new HashMap<List<Long>,NodeInfo>();
+    }
+    this.executor_node_port.put(key, val);
+  }
+
+  public Map<List<Long>,NodeInfo> get_executor_node_port() {
+    return this.executor_node_port;
+  }
+
+  public void set_executor_node_port(Map<List<Long>,NodeInfo> executor_node_port) {
+    this.executor_node_port = executor_node_port;
+  }
+
+  public void unset_executor_node_port() {
+    this.executor_node_port = null;
+  }
+
+  /** Returns true if field executor_node_port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_node_port() {
+    return this.executor_node_port != null;
+  }
+
+  public void set_executor_node_port_isSet(boolean value) {
+    if (!value) {
+      this.executor_node_port = null;
+    }
+  }
+
+  public int get_executor_start_time_secs_size() {
+    return (this.executor_start_time_secs == null) ? 0 : this.executor_start_time_secs.size();
+  }
+
+  public void put_to_executor_start_time_secs(List<Long> key, long val) {
+    if (this.executor_start_time_secs == null) {
+      this.executor_start_time_secs = new HashMap<List<Long>,Long>();
+    }
+    this.executor_start_time_secs.put(key, val);
+  }
+
+  public Map<List<Long>,Long> get_executor_start_time_secs() {
+    return this.executor_start_time_secs;
+  }
+
+  public void set_executor_start_time_secs(Map<List<Long>,Long> executor_start_time_secs) {
+    this.executor_start_time_secs = executor_start_time_secs;
+  }
+
+  public void unset_executor_start_time_secs() {
+    this.executor_start_time_secs = null;
+  }
+
+  /** Returns true if field executor_start_time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_start_time_secs() {
+    return this.executor_start_time_secs != null;
+  }
+
+  public void set_executor_start_time_secs_isSet(boolean value) {
+    if (!value) {
+      this.executor_start_time_secs = null;
+    }
+  }
+
+  public int get_worker_resources_size() {
+    return (this.worker_resources == null) ? 0 : this.worker_resources.size();
+  }
+
+  public void put_to_worker_resources(NodeInfo key, WorkerResources val) {
+    if (this.worker_resources == null) {
+      this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+    }
+    this.worker_resources.put(key, val);
+  }
+
+  public Map<NodeInfo,WorkerResources> get_worker_resources() {
+    return this.worker_resources;
+  }
+
+  public void set_worker_resources(Map<NodeInfo,WorkerResources> worker_resources) {
+    this.worker_resources = worker_resources;
+  }
+
+  public void unset_worker_resources() {
+    this.worker_resources = null;
+  }
+
+  /** Returns true if field worker_resources is set (has been assigned a value) and false otherwise */
+  public boolean is_set_worker_resources() {
+    return this.worker_resources != null;
+  }
+
+  public void set_worker_resources_isSet(boolean value) {
+    if (!value) {
+      this.worker_resources = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MASTER_CODE_DIR:
+      if (value == null) {
+        unset_master_code_dir();
+      } else {
+        set_master_code_dir((String)value);
+      }
+      break;
+
+    case NODE_HOST:
+      if (value == null) {
+        unset_node_host();
+      } else {
+        set_node_host((Map<String,String>)value);
+      }
+      break;
+
+    case EXECUTOR_NODE_PORT:
+      if (value == null) {
+        unset_executor_node_port();
+      } else {
+        set_executor_node_port((Map<List<Long>,NodeInfo>)value);
+      }
+      break;
+
+    case EXECUTOR_START_TIME_SECS:
+      if (value == null) {
+        unset_executor_start_time_secs();
+      } else {
+        set_executor_start_time_secs((Map<List<Long>,Long>)value);
+      }
+      break;
+
+    case WORKER_RESOURCES:
+      if (value == null) {
+        unset_worker_resources();
+      } else {
+        set_worker_resources((Map<NodeInfo,WorkerResources>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MASTER_CODE_DIR:
+      return get_master_code_dir();
+
+    case NODE_HOST:
+      return get_node_host();
+
+    case EXECUTOR_NODE_PORT:
+      return get_executor_node_port();
+
+    case EXECUTOR_START_TIME_SECS:
+      return get_executor_start_time_secs();
+
+    case WORKER_RESOURCES:
+      return get_worker_resources();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MASTER_CODE_DIR:
+      return is_set_master_code_dir();
+    case NODE_HOST:
+      return is_set_node_host();
+    case EXECUTOR_NODE_PORT:
+      return is_set_executor_node_port();
+    case EXECUTOR_START_TIME_SECS:
+      return is_set_executor_start_time_secs();
+    case WORKER_RESOURCES:
+      return is_set_worker_resources();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Assignment)
+      return this.equals((Assignment)that);
+    return false;
+  }
+
+  public boolean equals(Assignment that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_master_code_dir = true && this.is_set_master_code_dir();
+    boolean that_present_master_code_dir = true && that.is_set_master_code_dir();
+    if (this_present_master_code_dir || that_present_master_code_dir) {
+      if (!(this_present_master_code_dir && that_present_master_code_dir))
+        return false;
+      if (!this.master_code_dir.equals(that.master_code_dir))
+        return false;
+    }
+
+    boolean this_present_node_host = true && this.is_set_node_host();
+    boolean that_present_node_host = true && that.is_set_node_host();
+    if (this_present_node_host || that_present_node_host) {
+      if (!(this_present_node_host && that_present_node_host))
+        return false;
+      if (!this.node_host.equals(that.node_host))
+        return false;
+    }
+
+    boolean this_present_executor_node_port = true && this.is_set_executor_node_port();
+    boolean that_present_executor_node_port = true && that.is_set_executor_node_port();
+    if (this_present_executor_node_port || that_present_executor_node_port) {
+      if (!(this_present_executor_node_port && that_present_executor_node_port))
+        return false;
+      if (!this.executor_node_port.equals(that.executor_node_port))
+        return false;
+    }
+
+    boolean this_present_executor_start_time_secs = true && this.is_set_executor_start_time_secs();
+    boolean that_present_executor_start_time_secs = true && that.is_set_executor_start_time_secs();
+    if (this_present_executor_start_time_secs || that_present_executor_start_time_secs) {
+      if (!(this_present_executor_start_time_secs && that_present_executor_start_time_secs))
+        return false;
+      if (!this.executor_start_time_secs.equals(that.executor_start_time_secs))
+        return false;
+    }
+
+    boolean this_present_worker_resources = true && this.is_set_worker_resources();
+    boolean that_present_worker_resources = true && that.is_set_worker_resources();
+    if (this_present_worker_resources || that_present_worker_resources) {
+      if (!(this_present_worker_resources && that_present_worker_resources))
+        return false;
+      if (!this.worker_resources.equals(that.worker_resources))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_master_code_dir = true && (is_set_master_code_dir());
+    list.add(present_master_code_dir);
+    if (present_master_code_dir)
+      list.add(master_code_dir);
+
+    boolean present_node_host = true && (is_set_node_host());
+    list.add(present_node_host);
+    if (present_node_host)
+      list.add(node_host);
+
+    boolean present_executor_node_port = true && (is_set_executor_node_port());
+    list.add(present_executor_node_port);
+    if (present_executor_node_port)
+      list.add(executor_node_port);
+
+    boolean present_executor_start_time_secs = true && (is_set_executor_start_time_secs());
+    list.add(present_executor_start_time_secs);
+    if (present_executor_start_time_secs)
+      list.add(executor_start_time_secs);
+
+    boolean present_worker_resources = true && (is_set_worker_resources());
+    list.add(present_worker_resources);
+    if (present_worker_resources)
+      list.add(worker_resources);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(Assignment other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_master_code_dir()).compareTo(other.is_set_master_code_dir());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_master_code_dir()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.master_code_dir, other.master_code_dir);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_node_host()).compareTo(other.is_set_node_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_node_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node_host, other.node_host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executor_node_port()).compareTo(other.is_set_executor_node_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_node_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_node_port, other.executor_node_port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executor_start_time_secs()).compareTo(other.is_set_executor_start_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_start_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_start_time_secs, other.executor_start_time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_worker_resources()).compareTo(other.is_set_worker_resources());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_worker_resources()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.worker_resources, other.worker_resources);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Assignment(");
+    boolean first = true;
+
+    sb.append("master_code_dir:");
+    if (this.master_code_dir == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.master_code_dir);
+    }
+    first = false;
+    if (is_set_node_host()) {
+      if (!first) sb.append(", ");
+      sb.append("node_host:");
+      if (this.node_host == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.node_host);
+      }
+      first = false;
+    }
+    if (is_set_executor_node_port()) {
+      if (!first) sb.append(", ");
+      sb.append("executor_node_port:");
+      if (this.executor_node_port == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.executor_node_port);
+      }
+      first = false;
+    }
+    if (is_set_executor_start_time_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("executor_start_time_secs:");
+      if (this.executor_start_time_secs == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.executor_start_time_secs);
+      }
+      first = false;
+    }
+    if (is_set_worker_resources()) {
+      if (!first) sb.append(", ");
+      sb.append("worker_resources:");
+      if (this.worker_resources == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.worker_resources);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_master_code_dir()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'master_code_dir' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class AssignmentStandardSchemeFactory implements SchemeFactory {
+    public AssignmentStandardScheme getScheme() {
+      return new AssignmentStandardScheme();
+    }
+  }
+
+  private static class AssignmentStandardScheme extends StandardScheme<Assignment> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MASTER_CODE_DIR
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.master_code_dir = iprot.readString();
+              struct.set_master_code_dir_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NODE_HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map618 = iprot.readMapBegin();
+                struct.node_host = new HashMap<String,String>(2*_map618.size);
+                String _key619;
+                String _val620;
+                for (int _i621 = 0; _i621 < _map618.size; ++_i621)
+                {
+                  _key619 = iprot.readString();
+                  _val620 = iprot.readString();
+                  struct.node_host.put(_key619, _val620);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_node_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EXECUTOR_NODE_PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map622 = iprot.readMapBegin();
+                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map622.size);
+                List<Long> _key623;
+                NodeInfo _val624;
+                for (int _i625 = 0; _i625 < _map622.size; ++_i625)
+                {
+                  {
+                    org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
+                    _key623 = new ArrayList<Long>(_list626.size);
+                    long _elem627;
+                    for (int _i628 = 0; _i628 < _list626.size; ++_i628)
+                    {
+                      _elem627 = iprot.readI64();
+                      _key623.add(_elem627);
+                    }
+                    iprot.readListEnd();
+                  }
+                  _val624 = new NodeInfo();
+                  _val624.read(iprot);
+                  struct.executor_node_port.put(_key623, _val624);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executor_node_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // EXECUTOR_START_TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map629 = iprot.readMapBegin();
+                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map629.size);
+                List<Long> _key630;
+                long _val631;
+                for (int _i632 = 0; _i632 < _map629.size; ++_i632)
+                {
+                  {
+                    org.apache.thrift.protocol.TList _list633 = iprot.readListBegin();
+                    _key630 = new ArrayList<Long>(_list633.size);
+                    long _elem634;
+                    for (int _i635 = 0; _i635 < _list633.size; ++_i635)
+                    {
+                      _elem634 = iprot.readI64();
+                      _key630.add(_elem634);
+                    }
+                    iprot.readListEnd();
+                  }
+                  _val631 = iprot.readI64();
+                  struct.executor_start_time_secs.put(_key630, _val631);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executor_start_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // WORKER_RESOURCES
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map636 = iprot.readMapBegin();
+                struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map636.size);
+                NodeInfo _key637;
+                WorkerResources _val638;
+                for (int _i639 = 0; _i639 < _map636.size; ++_i639)
+                {
+                  _key637 = new NodeInfo();
+                  _key637.read(iprot);
+                  _val638 = new WorkerResources();
+                  _val638.read(iprot);
+                  struct.worker_resources.put(_key637, _val638);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_worker_resources_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.master_code_dir != null) {
+        oprot.writeFieldBegin(MASTER_CODE_DIR_FIELD_DESC);
+        oprot.writeString(struct.master_code_dir);
+        oprot.writeFieldEnd();
+      }
+      if (struct.node_host != null) {
+        if (struct.is_set_node_host()) {
+          oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.node_host.size()));
+            for (Map.Entry<String, String> _iter640 : struct.node_host.entrySet())
+            {
+              oprot.writeString(_iter640.getKey());
+              oprot.writeString(_iter640.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.executor_node_port != null) {
+        if (struct.is_set_executor_node_port()) {
+          oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, struct.executor_node_port.size()));
+            for (Map.Entry<List<Long>, NodeInfo> _iter641 : struct.executor_node_port.entrySet())
+            {
+              {
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter641.getKey().size()));
+                for (long _iter642 : _iter641.getKey())
+                {
+                  oprot.writeI64(_iter642);
+                }
+                oprot.writeListEnd();
+              }
+              _iter641.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.executor_start_time_secs != null) {
+        if (struct.is_set_executor_start_time_secs()) {
+          oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, struct.executor_start_time_secs.size()));
+            for (Map.Entry<List<Long>, Long> _iter643 : struct.executor_start_time_secs.entrySet())
+            {
+              {
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter643.getKey().size()));
+                for (long _iter644 : _iter643.getKey())
+                {
+                  oprot.writeI64(_iter644);
+                }
+                oprot.writeListEnd();
+              }
+              oprot.writeI64(_iter643.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.worker_resources != null) {
+        if (struct.is_set_worker_resources()) {
+          oprot.writeFieldBegin(WORKER_RESOURCES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.worker_resources.size()));
+            for (Map.Entry<NodeInfo, WorkerResources> _iter645 : struct.worker_resources.entrySet())
+            {
+              _iter645.getKey().write(oprot);
+              _iter645.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AssignmentTupleSchemeFactory implements SchemeFactory {
+    public AssignmentTupleScheme getScheme() {
+      return new AssignmentTupleScheme();
+    }
+  }
+
+  private static class AssignmentTupleScheme extends TupleScheme<Assignment> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, Assignment struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.master_code_dir);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_node_host()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_executor_node_port()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_executor_start_time_secs()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_worker_resources()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_node_host()) {
+        {
+          oprot.writeI32(struct.node_host.size());
+          for (Map.Entry<String, String> _iter646 : struct.node_host.entrySet())
+          {
+            oprot.writeString(_iter646.getKey());
+            oprot.writeString(_iter646.getValue());
+          }
+        }
+      }
+      if (struct.is_set_executor_node_port()) {
+        {
+          oprot.writeI32(struct.executor_node_port.size());
+          for (Map.Entry<List<Long>, NodeInfo> _iter647 : struct.executor_node_port.entrySet())
+          {
+            {
+              oprot.writeI32(_iter647.getKey().size());
+              for (long _iter648 : _iter647.getKey())
+              {
+                oprot.writeI64(_iter648);
+              }
+            }
+            _iter647.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_executor_start_time_secs()) {
+        {
+          oprot.writeI32(struct.executor_start_time_secs.size());
+          for (Map.Entry<List<Long>, Long> _iter649 : struct.executor_start_time_secs.entrySet())
+          {
+            {
+              oprot.writeI32(_iter649.getKey().size());
+              for (long _iter650 : _iter649.getKey())
+              {
+                oprot.writeI64(_iter650);
+              }
+            }
+            oprot.writeI64(_iter649.getValue());
+          }
+        }
+      }
+      if (struct.is_set_worker_resources()) {
+        {
+          oprot.writeI32(struct.worker_resources.size());
+          for (Map.Entry<NodeInfo, WorkerResources> _iter651 : struct.worker_resources.entrySet())
+          {
+            _iter651.getKey().write(oprot);
+            _iter651.getValue().write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, Assignment struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.master_code_dir = iprot.readString();
+      struct.set_master_code_dir_isSet(true);
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map652 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.node_host = new HashMap<String,String>(2*_map652.size);
+          String _key653;
+          String _val654;
+          for (int _i655 = 0; _i655 < _map652.size; ++_i655)
+          {
+            _key653 = iprot.readString();
+            _val654 = iprot.readString();
+            struct.node_host.put(_key653, _val654);
+          }
+        }
+        struct.set_node_host_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map656 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map656.size);
+          List<Long> _key657;
+          NodeInfo _val658;
+          for (int _i659 = 0; _i659 < _map656.size; ++_i659)
+          {
+            {
+              org.apache.thrift.protocol.TList _list660 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key657 = new ArrayList<Long>(_list660.size);
+              long _elem661;
+              for (int _i662 = 0; _i662 < _list660.size; ++_i662)
+              {
+                _elem661 = iprot.readI64();
+                _key657.add(_elem661);
+              }
+            }
+            _val658 = new NodeInfo();
+            _val658.read(iprot);
+            struct.executor_node_port.put(_key657, _val658);
+          }
+        }
+        struct.set_executor_node_port_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TMap _map663 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map663.size);
+          List<Long> _key664;
+          long _val665;
+          for (int _i666 = 0; _i666 < _map663.size; ++_i666)
+          {
+            {
+              org.apache.thrift.protocol.TList _list667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key664 = new ArrayList<Long>(_list667.size);
+              long _elem668;
+              for (int _i669 = 0; _i669 < _list667.size; ++_i669)
+              {
+                _elem668 = iprot.readI64();
+                _key664.add(_elem668);
+              }
+            }
+            _val665 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key664, _val665);
+          }
+        }
+        struct.set_executor_start_time_secs_isSet(true);
+      }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TMap _map670 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map670.size);
+          NodeInfo _key671;
+          WorkerResources _val672;
+          for (int _i673 = 0; _i673 < _map670.size; ++_i673)
+          {
+            _key671 = new NodeInfo();
+            _key671.read(iprot);
+            _val672 = new WorkerResources();
+            _val672.read(iprot);
+            struct.worker_resources.put(_key671, _val672);
+          }
+        }
+        struct.set_worker_resources_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/AuthorizationException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/AuthorizationException.java b/storm-client/src/jvm/org/apache/storm/generated/AuthorizationException.java
new file mode 100644
index 0000000..7358535
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/AuthorizationException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class AuthorizationException extends TException implements org.apache.thrift.TBase<AuthorizationException, AuthorizationException._Fields>, java.io.Serializable, Cloneable, Comparable<AuthorizationException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AuthorizationException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AuthorizationExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AuthorizationExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AuthorizationException.class, metaDataMap);
+  }
+
+  public AuthorizationException() {
+  }
+
+  public AuthorizationException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AuthorizationException(AuthorizationException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public AuthorizationException deepCopy() {
+    return new AuthorizationException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AuthorizationException)
+      return this.equals((AuthorizationException)that);
+    return false;
+  }
+
+  public boolean equals(AuthorizationException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AuthorizationException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AuthorizationException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class AuthorizationExceptionStandardSchemeFactory implements SchemeFactory {
+    public AuthorizationExceptionStandardScheme getScheme() {
+      return new AuthorizationExceptionStandardScheme();
+    }
+  }
+
+  private static class AuthorizationExceptionStandardScheme extends StandardScheme<AuthorizationException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AuthorizationException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, AuthorizationException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AuthorizationExceptionTupleSchemeFactory implements SchemeFactory {
+    public AuthorizationExceptionTupleScheme getScheme() {
+      return new AuthorizationExceptionTupleScheme();
+    }
+  }
+
+  private static class AuthorizationExceptionTupleScheme extends TupleScheme<AuthorizationException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AuthorizationException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, AuthorizationException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/BeginDownloadResult.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/BeginDownloadResult.java b/storm-client/src/jvm/org/apache/storm/generated/BeginDownloadResult.java
new file mode 100644
index 0000000..925f00b
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/BeginDownloadResult.java
@@ -0,0 +1,608 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class BeginDownloadResult implements org.apache.thrift.TBase<BeginDownloadResult, BeginDownloadResult._Fields>, java.io.Serializable, Cloneable, Comparable<BeginDownloadResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BeginDownloadResult");
+
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField SESSION_FIELD_DESC = new org.apache.thrift.protocol.TField("session", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField DATA_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("data_size", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new BeginDownloadResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new BeginDownloadResultTupleSchemeFactory());
+  }
+
+  private long version; // required
+  private String session; // required
+  private long data_size; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    VERSION((short)1, "version"),
+    SESSION((short)2, "session"),
+    DATA_SIZE((short)3, "data_size");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // VERSION
+          return VERSION;
+        case 2: // SESSION
+          return SESSION;
+        case 3: // DATA_SIZE
+          return DATA_SIZE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __VERSION_ISSET_ID = 0;
+  private static final int __DATA_SIZE_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.DATA_SIZE};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.SESSION, new org.apache.thrift.meta_data.FieldMetaData("session", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DATA_SIZE, new org.apache.thrift.meta_data.FieldMetaData("data_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(BeginDownloadResult.class, metaDataMap);
+  }
+
+  public BeginDownloadResult() {
+  }
+
+  public BeginDownloadResult(
+    long version,
+    String session)
+  {
+    this();
+    this.version = version;
+    set_version_isSet(true);
+    this.session = session;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public BeginDownloadResult(BeginDownloadResult other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.version = other.version;
+    if (other.is_set_session()) {
+      this.session = other.session;
+    }
+    this.data_size = other.data_size;
+  }
+
+  public BeginDownloadResult deepCopy() {
+    return new BeginDownloadResult(this);
+  }
+
+  @Override
+  public void clear() {
+    set_version_isSet(false);
+    this.version = 0;
+    this.session = null;
+    set_data_size_isSet(false);
+    this.data_size = 0;
+  }
+
+  public long get_version() {
+    return this.version;
+  }
+
+  public void set_version(long version) {
+    this.version = version;
+    set_version_isSet(true);
+  }
+
+  public void unset_version() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return EncodingUtils.testBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  public void set_version_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VERSION_ISSET_ID, value);
+  }
+
+  public String get_session() {
+    return this.session;
+  }
+
+  public void set_session(String session) {
+    this.session = session;
+  }
+
+  public void unset_session() {
+    this.session = null;
+  }
+
+  /** Returns true if field session is set (has been assigned a value) and false otherwise */
+  public boolean is_set_session() {
+    return this.session != null;
+  }
+
+  public void set_session_isSet(boolean value) {
+    if (!value) {
+      this.session = null;
+    }
+  }
+
+  public long get_data_size() {
+    return this.data_size;
+  }
+
+  public void set_data_size(long data_size) {
+    this.data_size = data_size;
+    set_data_size_isSet(true);
+  }
+
+  public void unset_data_size() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DATA_SIZE_ISSET_ID);
+  }
+
+  /** Returns true if field data_size is set (has been assigned a value) and false otherwise */
+  public boolean is_set_data_size() {
+    return EncodingUtils.testBit(__isset_bitfield, __DATA_SIZE_ISSET_ID);
+  }
+
+  public void set_data_size_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DATA_SIZE_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((Long)value);
+      }
+      break;
+
+    case SESSION:
+      if (value == null) {
+        unset_session();
+      } else {
+        set_session((String)value);
+      }
+      break;
+
+    case DATA_SIZE:
+      if (value == null) {
+        unset_data_size();
+      } else {
+        set_data_size((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case VERSION:
+      return get_version();
+
+    case SESSION:
+      return get_session();
+
+    case DATA_SIZE:
+      return get_data_size();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case VERSION:
+      return is_set_version();
+    case SESSION:
+      return is_set_session();
+    case DATA_SIZE:
+      return is_set_data_size();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof BeginDownloadResult)
+      return this.equals((BeginDownloadResult)that);
+    return false;
+  }
+
+  public boolean equals(BeginDownloadResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_version = true;
+    boolean that_present_version = true;
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (this.version != that.version)
+        return false;
+    }
+
+    boolean this_present_session = true && this.is_set_session();
+    boolean that_present_session = true && that.is_set_session();
+    if (this_present_session || that_present_session) {
+      if (!(this_present_session && that_present_session))
+        return false;
+      if (!this.session.equals(that.session))
+        return false;
+    }
+
+    boolean this_present_data_size = true && this.is_set_data_size();
+    boolean that_present_data_size = true && that.is_set_data_size();
+    if (this_present_data_size || that_present_data_size) {
+      if (!(this_present_data_size && that_present_data_size))
+        return false;
+      if (this.data_size != that.data_size)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_version = true;
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    boolean present_session = true && (is_set_session());
+    list.add(present_session);
+    if (present_session)
+      list.add(session);
+
+    boolean present_data_size = true && (is_set_data_size());
+    list.add(present_data_size);
+    if (present_data_size)
+      list.add(data_size);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(BeginDownloadResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_session()).compareTo(other.is_set_session());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_session()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.session, other.session);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_data_size()).compareTo(other.is_set_data_size());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_data_size()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data_size, other.data_size);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("BeginDownloadResult(");
+    boolean first = true;
+
+    sb.append("version:");
+    sb.append(this.version);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("session:");
+    if (this.session == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.session);
+    }
+    first = false;
+    if (is_set_data_size()) {
+      if (!first) sb.append(", ");
+      sb.append("data_size:");
+      sb.append(this.data_size);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_session()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'session' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class BeginDownloadResultStandardSchemeFactory implements SchemeFactory {
+    public BeginDownloadResultStandardScheme getScheme() {
+      return new BeginDownloadResultStandardScheme();
+    }
+  }
+
+  private static class BeginDownloadResultStandardScheme extends StandardScheme<BeginDownloadResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.version = iprot.readI64();
+              struct.set_version_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SESSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.session = iprot.readString();
+              struct.set_session_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // DATA_SIZE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.data_size = iprot.readI64();
+              struct.set_data_size_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(VERSION_FIELD_DESC);
+      oprot.writeI64(struct.version);
+      oprot.writeFieldEnd();
+      if (struct.session != null) {
+        oprot.writeFieldBegin(SESSION_FIELD_DESC);
+        oprot.writeString(struct.session);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_data_size()) {
+        oprot.writeFieldBegin(DATA_SIZE_FIELD_DESC);
+        oprot.writeI64(struct.data_size);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class BeginDownloadResultTupleSchemeFactory implements SchemeFactory {
+    public BeginDownloadResultTupleScheme getScheme() {
+      return new BeginDownloadResultTupleScheme();
+    }
+  }
+
+  private static class BeginDownloadResultTupleScheme extends TupleScheme<BeginDownloadResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.version);
+      oprot.writeString(struct.session);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_data_size()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_data_size()) {
+        oprot.writeI64(struct.data_size);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.version = iprot.readI64();
+      struct.set_version_isSet(true);
+      struct.session = iprot.readString();
+      struct.set_session_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.data_size = iprot.readI64();
+        struct.set_data_size_isSet(true);
+      }
+    }
+  }
+
+}
+


[36/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/Bolt.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/Bolt.java b/storm-client/src/jvm/org/apache/storm/generated/Bolt.java
new file mode 100644
index 0000000..878ebde
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/Bolt.java
@@ -0,0 +1,514 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class Bolt implements org.apache.thrift.TBase<Bolt, Bolt._Fields>, java.io.Serializable, Cloneable, Comparable<Bolt> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Bolt");
+
+  private static final org.apache.thrift.protocol.TField BOLT_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("bolt_object", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift.protocol.TField("common", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new BoltStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new BoltTupleSchemeFactory());
+  }
+
+  private ComponentObject bolt_object; // required
+  private ComponentCommon common; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    BOLT_OBJECT((short)1, "bolt_object"),
+    COMMON((short)2, "common");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // BOLT_OBJECT
+          return BOLT_OBJECT;
+        case 2: // COMMON
+          return COMMON;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.BOLT_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("bolt_object", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentObject.class)));
+    tmpMap.put(_Fields.COMMON, new org.apache.thrift.meta_data.FieldMetaData("common", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentCommon.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Bolt.class, metaDataMap);
+  }
+
+  public Bolt() {
+  }
+
+  public Bolt(
+    ComponentObject bolt_object,
+    ComponentCommon common)
+  {
+    this();
+    this.bolt_object = bolt_object;
+    this.common = common;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Bolt(Bolt other) {
+    if (other.is_set_bolt_object()) {
+      this.bolt_object = new ComponentObject(other.bolt_object);
+    }
+    if (other.is_set_common()) {
+      this.common = new ComponentCommon(other.common);
+    }
+  }
+
+  public Bolt deepCopy() {
+    return new Bolt(this);
+  }
+
+  @Override
+  public void clear() {
+    this.bolt_object = null;
+    this.common = null;
+  }
+
+  public ComponentObject get_bolt_object() {
+    return this.bolt_object;
+  }
+
+  public void set_bolt_object(ComponentObject bolt_object) {
+    this.bolt_object = bolt_object;
+  }
+
+  public void unset_bolt_object() {
+    this.bolt_object = null;
+  }
+
+  /** Returns true if field bolt_object is set (has been assigned a value) and false otherwise */
+  public boolean is_set_bolt_object() {
+    return this.bolt_object != null;
+  }
+
+  public void set_bolt_object_isSet(boolean value) {
+    if (!value) {
+      this.bolt_object = null;
+    }
+  }
+
+  public ComponentCommon get_common() {
+    return this.common;
+  }
+
+  public void set_common(ComponentCommon common) {
+    this.common = common;
+  }
+
+  public void unset_common() {
+    this.common = null;
+  }
+
+  /** Returns true if field common is set (has been assigned a value) and false otherwise */
+  public boolean is_set_common() {
+    return this.common != null;
+  }
+
+  public void set_common_isSet(boolean value) {
+    if (!value) {
+      this.common = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case BOLT_OBJECT:
+      if (value == null) {
+        unset_bolt_object();
+      } else {
+        set_bolt_object((ComponentObject)value);
+      }
+      break;
+
+    case COMMON:
+      if (value == null) {
+        unset_common();
+      } else {
+        set_common((ComponentCommon)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case BOLT_OBJECT:
+      return get_bolt_object();
+
+    case COMMON:
+      return get_common();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case BOLT_OBJECT:
+      return is_set_bolt_object();
+    case COMMON:
+      return is_set_common();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Bolt)
+      return this.equals((Bolt)that);
+    return false;
+  }
+
+  public boolean equals(Bolt that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_bolt_object = true && this.is_set_bolt_object();
+    boolean that_present_bolt_object = true && that.is_set_bolt_object();
+    if (this_present_bolt_object || that_present_bolt_object) {
+      if (!(this_present_bolt_object && that_present_bolt_object))
+        return false;
+      if (!this.bolt_object.equals(that.bolt_object))
+        return false;
+    }
+
+    boolean this_present_common = true && this.is_set_common();
+    boolean that_present_common = true && that.is_set_common();
+    if (this_present_common || that_present_common) {
+      if (!(this_present_common && that_present_common))
+        return false;
+      if (!this.common.equals(that.common))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_bolt_object = true && (is_set_bolt_object());
+    list.add(present_bolt_object);
+    if (present_bolt_object)
+      list.add(bolt_object);
+
+    boolean present_common = true && (is_set_common());
+    list.add(present_common);
+    if (present_common)
+      list.add(common);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(Bolt other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_bolt_object()).compareTo(other.is_set_bolt_object());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_bolt_object()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bolt_object, other.bolt_object);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common()).compareTo(other.is_set_common());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common, other.common);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Bolt(");
+    boolean first = true;
+
+    sb.append("bolt_object:");
+    if (this.bolt_object == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.bolt_object);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("common:");
+    if (this.common == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.common);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_bolt_object()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'bolt_object' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_common()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (common != null) {
+      common.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class BoltStandardSchemeFactory implements SchemeFactory {
+    public BoltStandardScheme getScheme() {
+      return new BoltStandardScheme();
+    }
+  }
+
+  private static class BoltStandardScheme extends StandardScheme<Bolt> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, Bolt struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // BOLT_OBJECT
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.bolt_object = new ComponentObject();
+              struct.bolt_object.read(iprot);
+              struct.set_bolt_object_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMMON
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.common = new ComponentCommon();
+              struct.common.read(iprot);
+              struct.set_common_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, Bolt struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.bolt_object != null) {
+        oprot.writeFieldBegin(BOLT_OBJECT_FIELD_DESC);
+        struct.bolt_object.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.common != null) {
+        oprot.writeFieldBegin(COMMON_FIELD_DESC);
+        struct.common.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class BoltTupleSchemeFactory implements SchemeFactory {
+    public BoltTupleScheme getScheme() {
+      return new BoltTupleScheme();
+    }
+  }
+
+  private static class BoltTupleScheme extends TupleScheme<Bolt> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, Bolt struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.bolt_object.write(oprot);
+      struct.common.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, Bolt struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.bolt_object = new ComponentObject();
+      struct.bolt_object.read(iprot);
+      struct.set_bolt_object_isSet(true);
+      struct.common = new ComponentCommon();
+      struct.common.read(iprot);
+      struct.set_common_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/BoltAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/BoltAggregateStats.java b/storm-client/src/jvm/org/apache/storm/generated/BoltAggregateStats.java
new file mode 100644
index 0000000..c5e6bf9
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/BoltAggregateStats.java
@@ -0,0 +1,704 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class BoltAggregateStats implements org.apache.thrift.TBase<BoltAggregateStats, BoltAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<BoltAggregateStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltAggregateStats");
+
+  private static final org.apache.thrift.protocol.TField EXECUTE_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("execute_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)1);
+  private static final org.apache.thrift.protocol.TField PROCESS_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("process_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)2);
+  private static final org.apache.thrift.protocol.TField EXECUTED_FIELD_DESC = new org.apache.thrift.protocol.TField("executed", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField CAPACITY_FIELD_DESC = new org.apache.thrift.protocol.TField("capacity", org.apache.thrift.protocol.TType.DOUBLE, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new BoltAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new BoltAggregateStatsTupleSchemeFactory());
+  }
+
+  private double execute_latency_ms; // optional
+  private double process_latency_ms; // optional
+  private long executed; // optional
+  private double capacity; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXECUTE_LATENCY_MS((short)1, "execute_latency_ms"),
+    PROCESS_LATENCY_MS((short)2, "process_latency_ms"),
+    EXECUTED((short)3, "executed"),
+    CAPACITY((short)4, "capacity");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXECUTE_LATENCY_MS
+          return EXECUTE_LATENCY_MS;
+        case 2: // PROCESS_LATENCY_MS
+          return PROCESS_LATENCY_MS;
+        case 3: // EXECUTED
+          return EXECUTED;
+        case 4: // CAPACITY
+          return CAPACITY;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __EXECUTE_LATENCY_MS_ISSET_ID = 0;
+  private static final int __PROCESS_LATENCY_MS_ISSET_ID = 1;
+  private static final int __EXECUTED_ISSET_ID = 2;
+  private static final int __CAPACITY_ISSET_ID = 3;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.EXECUTE_LATENCY_MS,_Fields.PROCESS_LATENCY_MS,_Fields.EXECUTED,_Fields.CAPACITY};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXECUTE_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("execute_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.PROCESS_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("process_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.EXECUTED, new org.apache.thrift.meta_data.FieldMetaData("executed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.CAPACITY, new org.apache.thrift.meta_data.FieldMetaData("capacity", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(BoltAggregateStats.class, metaDataMap);
+  }
+
+  public BoltAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public BoltAggregateStats(BoltAggregateStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.execute_latency_ms = other.execute_latency_ms;
+    this.process_latency_ms = other.process_latency_ms;
+    this.executed = other.executed;
+    this.capacity = other.capacity;
+  }
+
+  public BoltAggregateStats deepCopy() {
+    return new BoltAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    set_execute_latency_ms_isSet(false);
+    this.execute_latency_ms = 0.0;
+    set_process_latency_ms_isSet(false);
+    this.process_latency_ms = 0.0;
+    set_executed_isSet(false);
+    this.executed = 0;
+    set_capacity_isSet(false);
+    this.capacity = 0.0;
+  }
+
+  public double get_execute_latency_ms() {
+    return this.execute_latency_ms;
+  }
+
+  public void set_execute_latency_ms(double execute_latency_ms) {
+    this.execute_latency_ms = execute_latency_ms;
+    set_execute_latency_ms_isSet(true);
+  }
+
+  public void unset_execute_latency_ms() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID);
+  }
+
+  /** Returns true if field execute_latency_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_execute_latency_ms() {
+    return EncodingUtils.testBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID);
+  }
+
+  public void set_execute_latency_ms_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID, value);
+  }
+
+  public double get_process_latency_ms() {
+    return this.process_latency_ms;
+  }
+
+  public void set_process_latency_ms(double process_latency_ms) {
+    this.process_latency_ms = process_latency_ms;
+    set_process_latency_ms_isSet(true);
+  }
+
+  public void unset_process_latency_ms() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID);
+  }
+
+  /** Returns true if field process_latency_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_process_latency_ms() {
+    return EncodingUtils.testBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID);
+  }
+
+  public void set_process_latency_ms_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID, value);
+  }
+
+  public long get_executed() {
+    return this.executed;
+  }
+
+  public void set_executed(long executed) {
+    this.executed = executed;
+    set_executed_isSet(true);
+  }
+
+  public void unset_executed() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EXECUTED_ISSET_ID);
+  }
+
+  /** Returns true if field executed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executed() {
+    return EncodingUtils.testBit(__isset_bitfield, __EXECUTED_ISSET_ID);
+  }
+
+  public void set_executed_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXECUTED_ISSET_ID, value);
+  }
+
+  public double get_capacity() {
+    return this.capacity;
+  }
+
+  public void set_capacity(double capacity) {
+    this.capacity = capacity;
+    set_capacity_isSet(true);
+  }
+
+  public void unset_capacity() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CAPACITY_ISSET_ID);
+  }
+
+  /** Returns true if field capacity is set (has been assigned a value) and false otherwise */
+  public boolean is_set_capacity() {
+    return EncodingUtils.testBit(__isset_bitfield, __CAPACITY_ISSET_ID);
+  }
+
+  public void set_capacity_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CAPACITY_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXECUTE_LATENCY_MS:
+      if (value == null) {
+        unset_execute_latency_ms();
+      } else {
+        set_execute_latency_ms((Double)value);
+      }
+      break;
+
+    case PROCESS_LATENCY_MS:
+      if (value == null) {
+        unset_process_latency_ms();
+      } else {
+        set_process_latency_ms((Double)value);
+      }
+      break;
+
+    case EXECUTED:
+      if (value == null) {
+        unset_executed();
+      } else {
+        set_executed((Long)value);
+      }
+      break;
+
+    case CAPACITY:
+      if (value == null) {
+        unset_capacity();
+      } else {
+        set_capacity((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXECUTE_LATENCY_MS:
+      return get_execute_latency_ms();
+
+    case PROCESS_LATENCY_MS:
+      return get_process_latency_ms();
+
+    case EXECUTED:
+      return get_executed();
+
+    case CAPACITY:
+      return get_capacity();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXECUTE_LATENCY_MS:
+      return is_set_execute_latency_ms();
+    case PROCESS_LATENCY_MS:
+      return is_set_process_latency_ms();
+    case EXECUTED:
+      return is_set_executed();
+    case CAPACITY:
+      return is_set_capacity();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof BoltAggregateStats)
+      return this.equals((BoltAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(BoltAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_execute_latency_ms = true && this.is_set_execute_latency_ms();
+    boolean that_present_execute_latency_ms = true && that.is_set_execute_latency_ms();
+    if (this_present_execute_latency_ms || that_present_execute_latency_ms) {
+      if (!(this_present_execute_latency_ms && that_present_execute_latency_ms))
+        return false;
+      if (this.execute_latency_ms != that.execute_latency_ms)
+        return false;
+    }
+
+    boolean this_present_process_latency_ms = true && this.is_set_process_latency_ms();
+    boolean that_present_process_latency_ms = true && that.is_set_process_latency_ms();
+    if (this_present_process_latency_ms || that_present_process_latency_ms) {
+      if (!(this_present_process_latency_ms && that_present_process_latency_ms))
+        return false;
+      if (this.process_latency_ms != that.process_latency_ms)
+        return false;
+    }
+
+    boolean this_present_executed = true && this.is_set_executed();
+    boolean that_present_executed = true && that.is_set_executed();
+    if (this_present_executed || that_present_executed) {
+      if (!(this_present_executed && that_present_executed))
+        return false;
+      if (this.executed != that.executed)
+        return false;
+    }
+
+    boolean this_present_capacity = true && this.is_set_capacity();
+    boolean that_present_capacity = true && that.is_set_capacity();
+    if (this_present_capacity || that_present_capacity) {
+      if (!(this_present_capacity && that_present_capacity))
+        return false;
+      if (this.capacity != that.capacity)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_execute_latency_ms = true && (is_set_execute_latency_ms());
+    list.add(present_execute_latency_ms);
+    if (present_execute_latency_ms)
+      list.add(execute_latency_ms);
+
+    boolean present_process_latency_ms = true && (is_set_process_latency_ms());
+    list.add(present_process_latency_ms);
+    if (present_process_latency_ms)
+      list.add(process_latency_ms);
+
+    boolean present_executed = true && (is_set_executed());
+    list.add(present_executed);
+    if (present_executed)
+      list.add(executed);
+
+    boolean present_capacity = true && (is_set_capacity());
+    list.add(present_capacity);
+    if (present_capacity)
+      list.add(capacity);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(BoltAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_execute_latency_ms()).compareTo(other.is_set_execute_latency_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_execute_latency_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.execute_latency_ms, other.execute_latency_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_process_latency_ms()).compareTo(other.is_set_process_latency_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_process_latency_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.process_latency_ms, other.process_latency_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executed()).compareTo(other.is_set_executed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executed, other.executed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_capacity()).compareTo(other.is_set_capacity());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_capacity()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.capacity, other.capacity);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("BoltAggregateStats(");
+    boolean first = true;
+
+    if (is_set_execute_latency_ms()) {
+      sb.append("execute_latency_ms:");
+      sb.append(this.execute_latency_ms);
+      first = false;
+    }
+    if (is_set_process_latency_ms()) {
+      if (!first) sb.append(", ");
+      sb.append("process_latency_ms:");
+      sb.append(this.process_latency_ms);
+      first = false;
+    }
+    if (is_set_executed()) {
+      if (!first) sb.append(", ");
+      sb.append("executed:");
+      sb.append(this.executed);
+      first = false;
+    }
+    if (is_set_capacity()) {
+      if (!first) sb.append(", ");
+      sb.append("capacity:");
+      sb.append(this.capacity);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class BoltAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public BoltAggregateStatsStandardScheme getScheme() {
+      return new BoltAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class BoltAggregateStatsStandardScheme extends StandardScheme<BoltAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXECUTE_LATENCY_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.execute_latency_ms = iprot.readDouble();
+              struct.set_execute_latency_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PROCESS_LATENCY_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.process_latency_ms = iprot.readDouble();
+              struct.set_process_latency_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EXECUTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.executed = iprot.readI64();
+              struct.set_executed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // CAPACITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.capacity = iprot.readDouble();
+              struct.set_capacity_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_execute_latency_ms()) {
+        oprot.writeFieldBegin(EXECUTE_LATENCY_MS_FIELD_DESC);
+        oprot.writeDouble(struct.execute_latency_ms);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_process_latency_ms()) {
+        oprot.writeFieldBegin(PROCESS_LATENCY_MS_FIELD_DESC);
+        oprot.writeDouble(struct.process_latency_ms);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_executed()) {
+        oprot.writeFieldBegin(EXECUTED_FIELD_DESC);
+        oprot.writeI64(struct.executed);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_capacity()) {
+        oprot.writeFieldBegin(CAPACITY_FIELD_DESC);
+        oprot.writeDouble(struct.capacity);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class BoltAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public BoltAggregateStatsTupleScheme getScheme() {
+      return new BoltAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class BoltAggregateStatsTupleScheme extends TupleScheme<BoltAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_execute_latency_ms()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_process_latency_ms()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_executed()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_capacity()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_execute_latency_ms()) {
+        oprot.writeDouble(struct.execute_latency_ms);
+      }
+      if (struct.is_set_process_latency_ms()) {
+        oprot.writeDouble(struct.process_latency_ms);
+      }
+      if (struct.is_set_executed()) {
+        oprot.writeI64(struct.executed);
+      }
+      if (struct.is_set_capacity()) {
+        oprot.writeDouble(struct.capacity);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.execute_latency_ms = iprot.readDouble();
+        struct.set_execute_latency_ms_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.process_latency_ms = iprot.readDouble();
+        struct.set_process_latency_ms_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.executed = iprot.readI64();
+        struct.set_executed_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.capacity = iprot.readDouble();
+        struct.set_capacity_isSet(true);
+      }
+    }
+  }
+
+}
+


[34/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java b/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java
new file mode 100644
index 0000000..d974f1d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java
@@ -0,0 +1,768 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWorkerHeartbeat, ClusterWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterWorkerHeartbeat> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterWorkerHeartbeat");
+
+  private static final org.apache.thrift.protocol.TField STORM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storm_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_stats", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ClusterWorkerHeartbeatStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ClusterWorkerHeartbeatTupleSchemeFactory());
+  }
+
+  private String storm_id; // required
+  private Map<ExecutorInfo,ExecutorStats> executor_stats; // required
+  private int time_secs; // required
+  private int uptime_secs; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    STORM_ID((short)1, "storm_id"),
+    EXECUTOR_STATS((short)2, "executor_stats"),
+    TIME_SECS((short)3, "time_secs"),
+    UPTIME_SECS((short)4, "uptime_secs");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STORM_ID
+          return STORM_ID;
+        case 2: // EXECUTOR_STATS
+          return EXECUTOR_STATS;
+        case 3: // TIME_SECS
+          return TIME_SECS;
+        case 4: // UPTIME_SECS
+          return UPTIME_SECS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_SECS_ISSET_ID = 0;
+  private static final int __UPTIME_SECS_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STORM_ID, new org.apache.thrift.meta_data.FieldMetaData("storm_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EXECUTOR_STATS, new org.apache.thrift.meta_data.FieldMetaData("executor_stats", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorStats.class))));
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClusterWorkerHeartbeat.class, metaDataMap);
+  }
+
+  public ClusterWorkerHeartbeat() {
+  }
+
+  public ClusterWorkerHeartbeat(
+    String storm_id,
+    Map<ExecutorInfo,ExecutorStats> executor_stats,
+    int time_secs,
+    int uptime_secs)
+  {
+    this();
+    this.storm_id = storm_id;
+    this.executor_stats = executor_stats;
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ClusterWorkerHeartbeat(ClusterWorkerHeartbeat other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_storm_id()) {
+      this.storm_id = other.storm_id;
+    }
+    if (other.is_set_executor_stats()) {
+      Map<ExecutorInfo,ExecutorStats> __this__executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(other.executor_stats.size());
+      for (Map.Entry<ExecutorInfo, ExecutorStats> other_element : other.executor_stats.entrySet()) {
+
+        ExecutorInfo other_element_key = other_element.getKey();
+        ExecutorStats other_element_value = other_element.getValue();
+
+        ExecutorInfo __this__executor_stats_copy_key = new ExecutorInfo(other_element_key);
+
+        ExecutorStats __this__executor_stats_copy_value = new ExecutorStats(other_element_value);
+
+        __this__executor_stats.put(__this__executor_stats_copy_key, __this__executor_stats_copy_value);
+      }
+      this.executor_stats = __this__executor_stats;
+    }
+    this.time_secs = other.time_secs;
+    this.uptime_secs = other.uptime_secs;
+  }
+
+  public ClusterWorkerHeartbeat deepCopy() {
+    return new ClusterWorkerHeartbeat(this);
+  }
+
+  @Override
+  public void clear() {
+    this.storm_id = null;
+    this.executor_stats = null;
+    set_time_secs_isSet(false);
+    this.time_secs = 0;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+  }
+
+  public String get_storm_id() {
+    return this.storm_id;
+  }
+
+  public void set_storm_id(String storm_id) {
+    this.storm_id = storm_id;
+  }
+
+  public void unset_storm_id() {
+    this.storm_id = null;
+  }
+
+  /** Returns true if field storm_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_storm_id() {
+    return this.storm_id != null;
+  }
+
+  public void set_storm_id_isSet(boolean value) {
+    if (!value) {
+      this.storm_id = null;
+    }
+  }
+
+  public int get_executor_stats_size() {
+    return (this.executor_stats == null) ? 0 : this.executor_stats.size();
+  }
+
+  public void put_to_executor_stats(ExecutorInfo key, ExecutorStats val) {
+    if (this.executor_stats == null) {
+      this.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>();
+    }
+    this.executor_stats.put(key, val);
+  }
+
+  public Map<ExecutorInfo,ExecutorStats> get_executor_stats() {
+    return this.executor_stats;
+  }
+
+  public void set_executor_stats(Map<ExecutorInfo,ExecutorStats> executor_stats) {
+    this.executor_stats = executor_stats;
+  }
+
+  public void unset_executor_stats() {
+    this.executor_stats = null;
+  }
+
+  /** Returns true if field executor_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_stats() {
+    return this.executor_stats != null;
+  }
+
+  public void set_executor_stats_isSet(boolean value) {
+    if (!value) {
+      this.executor_stats = null;
+    }
+  }
+
+  public int get_time_secs() {
+    return this.time_secs;
+  }
+
+  public void set_time_secs(int time_secs) {
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  public void unset_time_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  public void set_time_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STORM_ID:
+      if (value == null) {
+        unset_storm_id();
+      } else {
+        set_storm_id((String)value);
+      }
+      break;
+
+    case EXECUTOR_STATS:
+      if (value == null) {
+        unset_executor_stats();
+      } else {
+        set_executor_stats((Map<ExecutorInfo,ExecutorStats>)value);
+      }
+      break;
+
+    case TIME_SECS:
+      if (value == null) {
+        unset_time_secs();
+      } else {
+        set_time_secs((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STORM_ID:
+      return get_storm_id();
+
+    case EXECUTOR_STATS:
+      return get_executor_stats();
+
+    case TIME_SECS:
+      return get_time_secs();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case STORM_ID:
+      return is_set_storm_id();
+    case EXECUTOR_STATS:
+      return is_set_executor_stats();
+    case TIME_SECS:
+      return is_set_time_secs();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ClusterWorkerHeartbeat)
+      return this.equals((ClusterWorkerHeartbeat)that);
+    return false;
+  }
+
+  public boolean equals(ClusterWorkerHeartbeat that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_storm_id = true && this.is_set_storm_id();
+    boolean that_present_storm_id = true && that.is_set_storm_id();
+    if (this_present_storm_id || that_present_storm_id) {
+      if (!(this_present_storm_id && that_present_storm_id))
+        return false;
+      if (!this.storm_id.equals(that.storm_id))
+        return false;
+    }
+
+    boolean this_present_executor_stats = true && this.is_set_executor_stats();
+    boolean that_present_executor_stats = true && that.is_set_executor_stats();
+    if (this_present_executor_stats || that_present_executor_stats) {
+      if (!(this_present_executor_stats && that_present_executor_stats))
+        return false;
+      if (!this.executor_stats.equals(that.executor_stats))
+        return false;
+    }
+
+    boolean this_present_time_secs = true;
+    boolean that_present_time_secs = true;
+    if (this_present_time_secs || that_present_time_secs) {
+      if (!(this_present_time_secs && that_present_time_secs))
+        return false;
+      if (this.time_secs != that.time_secs)
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_storm_id = true && (is_set_storm_id());
+    list.add(present_storm_id);
+    if (present_storm_id)
+      list.add(storm_id);
+
+    boolean present_executor_stats = true && (is_set_executor_stats());
+    list.add(present_executor_stats);
+    if (present_executor_stats)
+      list.add(executor_stats);
+
+    boolean present_time_secs = true;
+    list.add(present_time_secs);
+    if (present_time_secs)
+      list.add(time_secs);
+
+    boolean present_uptime_secs = true;
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ClusterWorkerHeartbeat other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_storm_id()).compareTo(other.is_set_storm_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_storm_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_id, other.storm_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executor_stats()).compareTo(other.is_set_executor_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_stats, other.executor_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ClusterWorkerHeartbeat(");
+    boolean first = true;
+
+    sb.append("storm_id:");
+    if (this.storm_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.storm_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executor_stats:");
+    if (this.executor_stats == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executor_stats);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("time_secs:");
+    sb.append(this.time_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_storm_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'storm_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executor_stats()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executor_stats' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_time_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ClusterWorkerHeartbeatStandardSchemeFactory implements SchemeFactory {
+    public ClusterWorkerHeartbeatStandardScheme getScheme() {
+      return new ClusterWorkerHeartbeatStandardScheme();
+    }
+  }
+
+  private static class ClusterWorkerHeartbeatStandardScheme extends StandardScheme<ClusterWorkerHeartbeat> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // STORM_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.storm_id = iprot.readString();
+              struct.set_storm_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // EXECUTOR_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map694 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map694.size);
+                ExecutorInfo _key695;
+                ExecutorStats _val696;
+                for (int _i697 = 0; _i697 < _map694.size; ++_i697)
+                {
+                  _key695 = new ExecutorInfo();
+                  _key695.read(iprot);
+                  _val696 = new ExecutorStats();
+                  _val696.read(iprot);
+                  struct.executor_stats.put(_key695, _val696);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executor_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.time_secs = iprot.readI32();
+              struct.set_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.storm_id != null) {
+        oprot.writeFieldBegin(STORM_ID_FIELD_DESC);
+        oprot.writeString(struct.storm_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.executor_stats != null) {
+        oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter698 : struct.executor_stats.entrySet())
+          {
+            _iter698.getKey().write(oprot);
+            _iter698.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.time_secs);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ClusterWorkerHeartbeatTupleSchemeFactory implements SchemeFactory {
+    public ClusterWorkerHeartbeatTupleScheme getScheme() {
+      return new ClusterWorkerHeartbeatTupleScheme();
+    }
+  }
+
+  private static class ClusterWorkerHeartbeatTupleScheme extends TupleScheme<ClusterWorkerHeartbeat> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.storm_id);
+      {
+        oprot.writeI32(struct.executor_stats.size());
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter699 : struct.executor_stats.entrySet())
+        {
+          _iter699.getKey().write(oprot);
+          _iter699.getValue().write(oprot);
+        }
+      }
+      oprot.writeI32(struct.time_secs);
+      oprot.writeI32(struct.uptime_secs);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.storm_id = iprot.readString();
+      struct.set_storm_id_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map700 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map700.size);
+        ExecutorInfo _key701;
+        ExecutorStats _val702;
+        for (int _i703 = 0; _i703 < _map700.size; ++_i703)
+        {
+          _key701 = new ExecutorInfo();
+          _key701.read(iprot);
+          _val702 = new ExecutorStats();
+          _val702.read(iprot);
+          struct.executor_stats.put(_key701, _val702);
+        }
+      }
+      struct.set_executor_stats_isSet(true);
+      struct.time_secs = iprot.readI32();
+      struct.set_time_secs_isSet(true);
+      struct.uptime_secs = iprot.readI32();
+      struct.set_uptime_secs_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/CommonAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/CommonAggregateStats.java b/storm-client/src/jvm/org/apache/storm/generated/CommonAggregateStats.java
new file mode 100644
index 0000000..ba35319
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/CommonAggregateStats.java
@@ -0,0 +1,1060 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class CommonAggregateStats implements org.apache.thrift.TBase<CommonAggregateStats, CommonAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<CommonAggregateStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CommonAggregateStats");
+
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("emitted", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("transferred", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("acked", org.apache.thrift.protocol.TType.I64, (short)5);
+  private static final org.apache.thrift.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("failed", org.apache.thrift.protocol.TType.I64, (short)6);
+  private static final org.apache.thrift.protocol.TField RESOURCES_MAP_FIELD_DESC = new org.apache.thrift.protocol.TField("resources_map", org.apache.thrift.protocol.TType.MAP, (short)7);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CommonAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CommonAggregateStatsTupleSchemeFactory());
+  }
+
+  private int num_executors; // optional
+  private int num_tasks; // optional
+  private long emitted; // optional
+  private long transferred; // optional
+  private long acked; // optional
+  private long failed; // optional
+  private Map<String,Double> resources_map; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NUM_EXECUTORS((short)1, "num_executors"),
+    NUM_TASKS((short)2, "num_tasks"),
+    EMITTED((short)3, "emitted"),
+    TRANSFERRED((short)4, "transferred"),
+    ACKED((short)5, "acked"),
+    FAILED((short)6, "failed"),
+    RESOURCES_MAP((short)7, "resources_map");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 2: // NUM_TASKS
+          return NUM_TASKS;
+        case 3: // EMITTED
+          return EMITTED;
+        case 4: // TRANSFERRED
+          return TRANSFERRED;
+        case 5: // ACKED
+          return ACKED;
+        case 6: // FAILED
+          return FAILED;
+        case 7: // RESOURCES_MAP
+          return RESOURCES_MAP;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_EXECUTORS_ISSET_ID = 0;
+  private static final int __NUM_TASKS_ISSET_ID = 1;
+  private static final int __EMITTED_ISSET_ID = 2;
+  private static final int __TRANSFERRED_ISSET_ID = 3;
+  private static final int __ACKED_ISSET_ID = 4;
+  private static final int __FAILED_ISSET_ID = 5;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NUM_EXECUTORS,_Fields.NUM_TASKS,_Fields.EMITTED,_Fields.TRANSFERRED,_Fields.ACKED,_Fields.FAILED,_Fields.RESOURCES_MAP};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.EMITTED, new org.apache.thrift.meta_data.FieldMetaData("emitted", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("transferred", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.ACKED, new org.apache.thrift.meta_data.FieldMetaData("acked", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.FAILED, new org.apache.thrift.meta_data.FieldMetaData("failed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.RESOURCES_MAP, new org.apache.thrift.meta_data.FieldMetaData("resources_map", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CommonAggregateStats.class, metaDataMap);
+  }
+
+  public CommonAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CommonAggregateStats(CommonAggregateStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.num_executors = other.num_executors;
+    this.num_tasks = other.num_tasks;
+    this.emitted = other.emitted;
+    this.transferred = other.transferred;
+    this.acked = other.acked;
+    this.failed = other.failed;
+    if (other.is_set_resources_map()) {
+      Map<String,Double> __this__resources_map = new HashMap<String,Double>(other.resources_map);
+      this.resources_map = __this__resources_map;
+    }
+  }
+
+  public CommonAggregateStats deepCopy() {
+    return new CommonAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    set_num_tasks_isSet(false);
+    this.num_tasks = 0;
+    set_emitted_isSet(false);
+    this.emitted = 0;
+    set_transferred_isSet(false);
+    this.transferred = 0;
+    set_acked_isSet(false);
+    this.acked = 0;
+    set_failed_isSet(false);
+    this.failed = 0;
+    this.resources_map = null;
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value);
+  }
+
+  public long get_emitted() {
+    return this.emitted;
+  }
+
+  public void set_emitted(long emitted) {
+    this.emitted = emitted;
+    set_emitted_isSet(true);
+  }
+
+  public void unset_emitted() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EMITTED_ISSET_ID);
+  }
+
+  /** Returns true if field emitted is set (has been assigned a value) and false otherwise */
+  public boolean is_set_emitted() {
+    return EncodingUtils.testBit(__isset_bitfield, __EMITTED_ISSET_ID);
+  }
+
+  public void set_emitted_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EMITTED_ISSET_ID, value);
+  }
+
+  public long get_transferred() {
+    return this.transferred;
+  }
+
+  public void set_transferred(long transferred) {
+    this.transferred = transferred;
+    set_transferred_isSet(true);
+  }
+
+  public void unset_transferred() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TRANSFERRED_ISSET_ID);
+  }
+
+  /** Returns true if field transferred is set (has been assigned a value) and false otherwise */
+  public boolean is_set_transferred() {
+    return EncodingUtils.testBit(__isset_bitfield, __TRANSFERRED_ISSET_ID);
+  }
+
+  public void set_transferred_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TRANSFERRED_ISSET_ID, value);
+  }
+
+  public long get_acked() {
+    return this.acked;
+  }
+
+  public void set_acked(long acked) {
+    this.acked = acked;
+    set_acked_isSet(true);
+  }
+
+  public void unset_acked() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ACKED_ISSET_ID);
+  }
+
+  /** Returns true if field acked is set (has been assigned a value) and false otherwise */
+  public boolean is_set_acked() {
+    return EncodingUtils.testBit(__isset_bitfield, __ACKED_ISSET_ID);
+  }
+
+  public void set_acked_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ACKED_ISSET_ID, value);
+  }
+
+  public long get_failed() {
+    return this.failed;
+  }
+
+  public void set_failed(long failed) {
+    this.failed = failed;
+    set_failed_isSet(true);
+  }
+
+  public void unset_failed() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FAILED_ISSET_ID);
+  }
+
+  /** Returns true if field failed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_failed() {
+    return EncodingUtils.testBit(__isset_bitfield, __FAILED_ISSET_ID);
+  }
+
+  public void set_failed_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FAILED_ISSET_ID, value);
+  }
+
+  public int get_resources_map_size() {
+    return (this.resources_map == null) ? 0 : this.resources_map.size();
+  }
+
+  public void put_to_resources_map(String key, double val) {
+    if (this.resources_map == null) {
+      this.resources_map = new HashMap<String,Double>();
+    }
+    this.resources_map.put(key, val);
+  }
+
+  public Map<String,Double> get_resources_map() {
+    return this.resources_map;
+  }
+
+  public void set_resources_map(Map<String,Double> resources_map) {
+    this.resources_map = resources_map;
+  }
+
+  public void unset_resources_map() {
+    this.resources_map = null;
+  }
+
+  /** Returns true if field resources_map is set (has been assigned a value) and false otherwise */
+  public boolean is_set_resources_map() {
+    return this.resources_map != null;
+  }
+
+  public void set_resources_map_isSet(boolean value) {
+    if (!value) {
+      this.resources_map = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case EMITTED:
+      if (value == null) {
+        unset_emitted();
+      } else {
+        set_emitted((Long)value);
+      }
+      break;
+
+    case TRANSFERRED:
+      if (value == null) {
+        unset_transferred();
+      } else {
+        set_transferred((Long)value);
+      }
+      break;
+
+    case ACKED:
+      if (value == null) {
+        unset_acked();
+      } else {
+        set_acked((Long)value);
+      }
+      break;
+
+    case FAILED:
+      if (value == null) {
+        unset_failed();
+      } else {
+        set_failed((Long)value);
+      }
+      break;
+
+    case RESOURCES_MAP:
+      if (value == null) {
+        unset_resources_map();
+      } else {
+        set_resources_map((Map<String,Double>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case NUM_TASKS:
+      return get_num_tasks();
+
+    case EMITTED:
+      return get_emitted();
+
+    case TRANSFERRED:
+      return get_transferred();
+
+    case ACKED:
+      return get_acked();
+
+    case FAILED:
+      return get_failed();
+
+    case RESOURCES_MAP:
+      return get_resources_map();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case EMITTED:
+      return is_set_emitted();
+    case TRANSFERRED:
+      return is_set_transferred();
+    case ACKED:
+      return is_set_acked();
+    case FAILED:
+      return is_set_failed();
+    case RESOURCES_MAP:
+      return is_set_resources_map();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CommonAggregateStats)
+      return this.equals((CommonAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(CommonAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_num_tasks = true && this.is_set_num_tasks();
+    boolean that_present_num_tasks = true && that.is_set_num_tasks();
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_emitted = true && this.is_set_emitted();
+    boolean that_present_emitted = true && that.is_set_emitted();
+    if (this_present_emitted || that_present_emitted) {
+      if (!(this_present_emitted && that_present_emitted))
+        return false;
+      if (this.emitted != that.emitted)
+        return false;
+    }
+
+    boolean this_present_transferred = true && this.is_set_transferred();
+    boolean that_present_transferred = true && that.is_set_transferred();
+    if (this_present_transferred || that_present_transferred) {
+      if (!(this_present_transferred && that_present_transferred))
+        return false;
+      if (this.transferred != that.transferred)
+        return false;
+    }
+
+    boolean this_present_acked = true && this.is_set_acked();
+    boolean that_present_acked = true && that.is_set_acked();
+    if (this_present_acked || that_present_acked) {
+      if (!(this_present_acked && that_present_acked))
+        return false;
+      if (this.acked != that.acked)
+        return false;
+    }
+
+    boolean this_present_failed = true && this.is_set_failed();
+    boolean that_present_failed = true && that.is_set_failed();
+    if (this_present_failed || that_present_failed) {
+      if (!(this_present_failed && that_present_failed))
+        return false;
+      if (this.failed != that.failed)
+        return false;
+    }
+
+    boolean this_present_resources_map = true && this.is_set_resources_map();
+    boolean that_present_resources_map = true && that.is_set_resources_map();
+    if (this_present_resources_map || that_present_resources_map) {
+      if (!(this_present_resources_map && that_present_resources_map))
+        return false;
+      if (!this.resources_map.equals(that.resources_map))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_num_tasks = true && (is_set_num_tasks());
+    list.add(present_num_tasks);
+    if (present_num_tasks)
+      list.add(num_tasks);
+
+    boolean present_emitted = true && (is_set_emitted());
+    list.add(present_emitted);
+    if (present_emitted)
+      list.add(emitted);
+
+    boolean present_transferred = true && (is_set_transferred());
+    list.add(present_transferred);
+    if (present_transferred)
+      list.add(transferred);
+
+    boolean present_acked = true && (is_set_acked());
+    list.add(present_acked);
+    if (present_acked)
+      list.add(acked);
+
+    boolean present_failed = true && (is_set_failed());
+    list.add(present_failed);
+    if (present_failed)
+      list.add(failed);
+
+    boolean present_resources_map = true && (is_set_resources_map());
+    list.add(present_resources_map);
+    if (present_resources_map)
+      list.add(resources_map);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(CommonAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_emitted()).compareTo(other.is_set_emitted());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_emitted()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.emitted, other.emitted);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_transferred()).compareTo(other.is_set_transferred());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_transferred()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.transferred, other.transferred);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_acked()).compareTo(other.is_set_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acked()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acked, other.acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_failed()).compareTo(other.is_set_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_failed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.failed, other.failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_resources_map()).compareTo(other.is_set_resources_map());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_resources_map()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resources_map, other.resources_map);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("CommonAggregateStats(");
+    boolean first = true;
+
+    if (is_set_num_executors()) {
+      sb.append("num_executors:");
+      sb.append(this.num_executors);
+      first = false;
+    }
+    if (is_set_num_tasks()) {
+      if (!first) sb.append(", ");
+      sb.append("num_tasks:");
+      sb.append(this.num_tasks);
+      first = false;
+    }
+    if (is_set_emitted()) {
+      if (!first) sb.append(", ");
+      sb.append("emitted:");
+      sb.append(this.emitted);
+      first = false;
+    }
+    if (is_set_transferred()) {
+      if (!first) sb.append(", ");
+      sb.append("transferred:");
+      sb.append(this.transferred);
+      first = false;
+    }
+    if (is_set_acked()) {
+      if (!first) sb.append(", ");
+      sb.append("acked:");
+      sb.append(this.acked);
+      first = false;
+    }
+    if (is_set_failed()) {
+      if (!first) sb.append(", ");
+      sb.append("failed:");
+      sb.append(this.failed);
+      first = false;
+    }
+    if (is_set_resources_map()) {
+      if (!first) sb.append(", ");
+      sb.append("resources_map:");
+      if (this.resources_map == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.resources_map);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CommonAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public CommonAggregateStatsStandardScheme getScheme() {
+      return new CommonAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class CommonAggregateStatsStandardScheme extends StandardScheme<CommonAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_tasks = iprot.readI32();
+              struct.set_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EMITTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.emitted = iprot.readI64();
+              struct.set_emitted_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TRANSFERRED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.transferred = iprot.readI64();
+              struct.set_transferred_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // ACKED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.acked = iprot.readI64();
+              struct.set_acked_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // FAILED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.failed = iprot.readI64();
+              struct.set_failed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // RESOURCES_MAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map368 = iprot.readMapBegin();
+                struct.resources_map = new HashMap<String,Double>(2*_map368.size);
+                String _key369;
+                double _val370;
+                for (int _i371 = 0; _i371 < _map368.size; ++_i371)
+                {
+                  _key369 = iprot.readString();
+                  _val370 = iprot.readDouble();
+                  struct.resources_map.put(_key369, _val370);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_resources_map_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_num_executors()) {
+        oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+        oprot.writeI32(struct.num_executors);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+        oprot.writeI32(struct.num_tasks);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_emitted()) {
+        oprot.writeFieldBegin(EMITTED_FIELD_DESC);
+        oprot.writeI64(struct.emitted);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_transferred()) {
+        oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC);
+        oprot.writeI64(struct.transferred);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_acked()) {
+        oprot.writeFieldBegin(ACKED_FIELD_DESC);
+        oprot.writeI64(struct.acked);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_failed()) {
+        oprot.writeFieldBegin(FAILED_FIELD_DESC);
+        oprot.writeI64(struct.failed);
+        oprot.writeFieldEnd();
+      }
+      if (struct.resources_map != null) {
+        if (struct.is_set_resources_map()) {
+          oprot.writeFieldBegin(RESOURCES_MAP_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.resources_map.size()));
+            for (Map.Entry<String, Double> _iter372 : struct.resources_map.entrySet())
+            {
+              oprot.writeString(_iter372.getKey());
+              oprot.writeDouble(_iter372.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CommonAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public CommonAggregateStatsTupleScheme getScheme() {
+      return new CommonAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class CommonAggregateStatsTupleScheme extends TupleScheme<CommonAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_num_executors()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_num_tasks()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_emitted()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_transferred()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_acked()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_failed()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_resources_map()) {
+        optionals.set(6);
+      }
+      oprot.writeBitSet(optionals, 7);
+      if (struct.is_set_num_executors()) {
+        oprot.writeI32(struct.num_executors);
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeI32(struct.num_tasks);
+      }
+      if (struct.is_set_emitted()) {
+        oprot.writeI64(struct.emitted);
+      }
+      if (struct.is_set_transferred()) {
+        oprot.writeI64(struct.transferred);
+      }
+      if (struct.is_set_acked()) {
+        oprot.writeI64(struct.acked);
+      }
+      if (struct.is_set_failed()) {
+        oprot.writeI64(struct.failed);
+      }
+      if (struct.is_set_resources_map()) {
+        {
+          oprot.writeI32(struct.resources_map.size());
+          for (Map.Entry<String, Double> _iter373 : struct.resources_map.entrySet())
+          {
+            oprot.writeString(_iter373.getKey());
+            oprot.writeDouble(_iter373.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(7);
+      if (incoming.get(0)) {
+        struct.num_executors = iprot.readI32();
+        struct.set_num_executors_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.num_tasks = iprot.readI32();
+        struct.set_num_tasks_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.emitted = iprot.readI64();
+        struct.set_emitted_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.transferred = iprot.readI64();
+        struct.set_transferred_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.acked = iprot.readI64();
+        struct.set_acked_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.failed = iprot.readI64();
+        struct.set_failed_isSet(true);
+      }
+      if (incoming.get(6)) {
+        {
+          org.apache.thrift.protocol.TMap _map374 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.resources_map = new HashMap<String,Double>(2*_map374.size);
+          String _key375;
+          double _val376;
+          for (int _i377 = 0; _i377 < _map374.size; ++_i377)
+          {
+            _key375 = iprot.readString();
+            _val376 = iprot.readDouble();
+            struct.resources_map.put(_key375, _val376);
+          }
+        }
+        struct.set_resources_map_isSet(true);
+      }
+    }
+  }
+
+}
+


[31/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ComponentType.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ComponentType.java b/storm-client/src/jvm/org/apache/storm/generated/ComponentType.java
new file mode 100644
index 0000000..7a9fd66
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ComponentType.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum ComponentType implements org.apache.thrift.TEnum {
+  BOLT(1),
+  SPOUT(2);
+
+  private final int value;
+
+  private ComponentType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static ComponentType findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return BOLT;
+      case 2:
+        return SPOUT;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/Credentials.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/Credentials.java b/storm-client/src/jvm/org/apache/storm/generated/Credentials.java
new file mode 100644
index 0000000..231af06
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/Credentials.java
@@ -0,0 +1,458 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class Credentials implements org.apache.thrift.TBase<Credentials, Credentials._Fields>, java.io.Serializable, Cloneable, Comparable<Credentials> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials");
+
+  private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.MAP, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CredentialsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CredentialsTupleSchemeFactory());
+  }
+
+  private Map<String,String> creds; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    CREDS((short)1, "creds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CREDS
+          return CREDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Credentials.class, metaDataMap);
+  }
+
+  public Credentials() {
+  }
+
+  public Credentials(
+    Map<String,String> creds)
+  {
+    this();
+    this.creds = creds;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Credentials(Credentials other) {
+    if (other.is_set_creds()) {
+      Map<String,String> __this__creds = new HashMap<String,String>(other.creds);
+      this.creds = __this__creds;
+    }
+  }
+
+  public Credentials deepCopy() {
+    return new Credentials(this);
+  }
+
+  @Override
+  public void clear() {
+    this.creds = null;
+  }
+
+  public int get_creds_size() {
+    return (this.creds == null) ? 0 : this.creds.size();
+  }
+
+  public void put_to_creds(String key, String val) {
+    if (this.creds == null) {
+      this.creds = new HashMap<String,String>();
+    }
+    this.creds.put(key, val);
+  }
+
+  public Map<String,String> get_creds() {
+    return this.creds;
+  }
+
+  public void set_creds(Map<String,String> creds) {
+    this.creds = creds;
+  }
+
+  public void unset_creds() {
+    this.creds = null;
+  }
+
+  /** Returns true if field creds is set (has been assigned a value) and false otherwise */
+  public boolean is_set_creds() {
+    return this.creds != null;
+  }
+
+  public void set_creds_isSet(boolean value) {
+    if (!value) {
+      this.creds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CREDS:
+      if (value == null) {
+        unset_creds();
+      } else {
+        set_creds((Map<String,String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CREDS:
+      return get_creds();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case CREDS:
+      return is_set_creds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Credentials)
+      return this.equals((Credentials)that);
+    return false;
+  }
+
+  public boolean equals(Credentials that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_creds = true && this.is_set_creds();
+    boolean that_present_creds = true && that.is_set_creds();
+    if (this_present_creds || that_present_creds) {
+      if (!(this_present_creds && that_present_creds))
+        return false;
+      if (!this.creds.equals(that.creds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_creds = true && (is_set_creds());
+    list.add(present_creds);
+    if (present_creds)
+      list.add(creds);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(Credentials other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_creds()).compareTo(other.is_set_creds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_creds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, other.creds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Credentials(");
+    boolean first = true;
+
+    sb.append("creds:");
+    if (this.creds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.creds);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_creds()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'creds' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CredentialsStandardSchemeFactory implements SchemeFactory {
+    public CredentialsStandardScheme getScheme() {
+      return new CredentialsStandardScheme();
+    }
+  }
+
+  private static class CredentialsStandardScheme extends StandardScheme<Credentials> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, Credentials struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // CREDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map548 = iprot.readMapBegin();
+                struct.creds = new HashMap<String,String>(2*_map548.size);
+                String _key549;
+                String _val550;
+                for (int _i551 = 0; _i551 < _map548.size; ++_i551)
+                {
+                  _key549 = iprot.readString();
+                  _val550 = iprot.readString();
+                  struct.creds.put(_key549, _val550);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_creds_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, Credentials struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.creds != null) {
+        oprot.writeFieldBegin(CREDS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.creds.size()));
+          for (Map.Entry<String, String> _iter552 : struct.creds.entrySet())
+          {
+            oprot.writeString(_iter552.getKey());
+            oprot.writeString(_iter552.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CredentialsTupleSchemeFactory implements SchemeFactory {
+    public CredentialsTupleScheme getScheme() {
+      return new CredentialsTupleScheme();
+    }
+  }
+
+  private static class CredentialsTupleScheme extends TupleScheme<Credentials> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.creds.size());
+        for (Map.Entry<String, String> _iter553 : struct.creds.entrySet())
+        {
+          oprot.writeString(_iter553.getKey());
+          oprot.writeString(_iter553.getValue());
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map554 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.creds = new HashMap<String,String>(2*_map554.size);
+        String _key555;
+        String _val556;
+        for (int _i557 = 0; _i557 < _map554.size; ++_i557)
+        {
+          _key555 = iprot.readString();
+          _val556 = iprot.readString();
+          struct.creds.put(_key555, _val556);
+        }
+      }
+      struct.set_creds_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/DRPCExceptionType.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/DRPCExceptionType.java b/storm-client/src/jvm/org/apache/storm/generated/DRPCExceptionType.java
new file mode 100644
index 0000000..2ccf1b1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/DRPCExceptionType.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum DRPCExceptionType implements org.apache.thrift.TEnum {
+  INTERNAL_ERROR(0),
+  SERVER_SHUTDOWN(1),
+  SERVER_TIMEOUT(2),
+  FAILED_REQUEST(3);
+
+  private final int value;
+
+  private DRPCExceptionType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static DRPCExceptionType findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return INTERNAL_ERROR;
+      case 1:
+        return SERVER_SHUTDOWN;
+      case 2:
+        return SERVER_TIMEOUT;
+      case 3:
+        return FAILED_REQUEST;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/DRPCExecutionException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/DRPCExecutionException.java b/storm-client/src/jvm/org/apache/storm/generated/DRPCExecutionException.java
new file mode 100644
index 0000000..fd701ff
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/DRPCExecutionException.java
@@ -0,0 +1,528 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class DRPCExecutionException extends TException implements org.apache.thrift.TBase<DRPCExecutionException, DRPCExecutionException._Fields>, java.io.Serializable, Cloneable, Comparable<DRPCExecutionException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DRPCExecutionException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DRPCExecutionExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DRPCExecutionExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+  private DRPCExceptionType type; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg"),
+    /**
+     * 
+     * @see DRPCExceptionType
+     */
+    TYPE((short)2, "type");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        case 2: // TYPE
+          return TYPE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.TYPE};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, DRPCExceptionType.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DRPCExecutionException.class, metaDataMap);
+  }
+
+  public DRPCExecutionException() {
+  }
+
+  public DRPCExecutionException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DRPCExecutionException(DRPCExecutionException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+    if (other.is_set_type()) {
+      this.type = other.type;
+    }
+  }
+
+  public DRPCExecutionException deepCopy() {
+    return new DRPCExecutionException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+    this.type = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  /**
+   * 
+   * @see DRPCExceptionType
+   */
+  public DRPCExceptionType get_type() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see DRPCExceptionType
+   */
+  public void set_type(DRPCExceptionType type) {
+    this.type = type;
+  }
+
+  public void unset_type() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_type() {
+    return this.type != null;
+  }
+
+  public void set_type_isSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    case TYPE:
+      if (value == null) {
+        unset_type();
+      } else {
+        set_type((DRPCExceptionType)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    case TYPE:
+      return get_type();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    case TYPE:
+      return is_set_type();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DRPCExecutionException)
+      return this.equals((DRPCExecutionException)that);
+    return false;
+  }
+
+  public boolean equals(DRPCExecutionException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    boolean this_present_type = true && this.is_set_type();
+    boolean that_present_type = true && that.is_set_type();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    boolean present_type = true && (is_set_type());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DRPCExecutionException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("DRPCExecutionException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    if (is_set_type()) {
+      if (!first) sb.append(", ");
+      sb.append("type:");
+      if (this.type == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.type);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class DRPCExecutionExceptionStandardSchemeFactory implements SchemeFactory {
+    public DRPCExecutionExceptionStandardScheme getScheme() {
+      return new DRPCExecutionExceptionStandardScheme();
+    }
+  }
+
+  private static class DRPCExecutionExceptionStandardScheme extends StandardScheme<DRPCExecutionException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DRPCExecutionException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = org.apache.storm.generated.DRPCExceptionType.findByValue(iprot.readI32());
+              struct.set_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, DRPCExecutionException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      if (struct.type != null) {
+        if (struct.is_set_type()) {
+          oprot.writeFieldBegin(TYPE_FIELD_DESC);
+          oprot.writeI32(struct.type.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DRPCExecutionExceptionTupleSchemeFactory implements SchemeFactory {
+    public DRPCExecutionExceptionTupleScheme getScheme() {
+      return new DRPCExecutionExceptionTupleScheme();
+    }
+  }
+
+  private static class DRPCExecutionExceptionTupleScheme extends TupleScheme<DRPCExecutionException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DRPCExecutionException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_type()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_type()) {
+        oprot.writeI32(struct.type.getValue());
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DRPCExecutionException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.type = org.apache.storm.generated.DRPCExceptionType.findByValue(iprot.readI32());
+        struct.set_type_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/DRPCRequest.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/DRPCRequest.java b/storm-client/src/jvm/org/apache/storm/generated/DRPCRequest.java
new file mode 100644
index 0000000..33d85c2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/DRPCRequest.java
@@ -0,0 +1,507 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class DRPCRequest implements org.apache.thrift.TBase<DRPCRequest, DRPCRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DRPCRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DRPCRequest");
+
+  private static final org.apache.thrift.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift.protocol.TField("func_args", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField REQUEST_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("request_id", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DRPCRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DRPCRequestTupleSchemeFactory());
+  }
+
+  private String func_args; // required
+  private String request_id; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FUNC_ARGS((short)1, "func_args"),
+    REQUEST_ID((short)2, "request_id");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FUNC_ARGS
+          return FUNC_ARGS;
+        case 2: // REQUEST_ID
+          return REQUEST_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift.meta_data.FieldMetaData("func_args", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.REQUEST_ID, new org.apache.thrift.meta_data.FieldMetaData("request_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DRPCRequest.class, metaDataMap);
+  }
+
+  public DRPCRequest() {
+  }
+
+  public DRPCRequest(
+    String func_args,
+    String request_id)
+  {
+    this();
+    this.func_args = func_args;
+    this.request_id = request_id;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DRPCRequest(DRPCRequest other) {
+    if (other.is_set_func_args()) {
+      this.func_args = other.func_args;
+    }
+    if (other.is_set_request_id()) {
+      this.request_id = other.request_id;
+    }
+  }
+
+  public DRPCRequest deepCopy() {
+    return new DRPCRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.func_args = null;
+    this.request_id = null;
+  }
+
+  public String get_func_args() {
+    return this.func_args;
+  }
+
+  public void set_func_args(String func_args) {
+    this.func_args = func_args;
+  }
+
+  public void unset_func_args() {
+    this.func_args = null;
+  }
+
+  /** Returns true if field func_args is set (has been assigned a value) and false otherwise */
+  public boolean is_set_func_args() {
+    return this.func_args != null;
+  }
+
+  public void set_func_args_isSet(boolean value) {
+    if (!value) {
+      this.func_args = null;
+    }
+  }
+
+  public String get_request_id() {
+    return this.request_id;
+  }
+
+  public void set_request_id(String request_id) {
+    this.request_id = request_id;
+  }
+
+  public void unset_request_id() {
+    this.request_id = null;
+  }
+
+  /** Returns true if field request_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_request_id() {
+    return this.request_id != null;
+  }
+
+  public void set_request_id_isSet(boolean value) {
+    if (!value) {
+      this.request_id = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FUNC_ARGS:
+      if (value == null) {
+        unset_func_args();
+      } else {
+        set_func_args((String)value);
+      }
+      break;
+
+    case REQUEST_ID:
+      if (value == null) {
+        unset_request_id();
+      } else {
+        set_request_id((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FUNC_ARGS:
+      return get_func_args();
+
+    case REQUEST_ID:
+      return get_request_id();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case FUNC_ARGS:
+      return is_set_func_args();
+    case REQUEST_ID:
+      return is_set_request_id();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DRPCRequest)
+      return this.equals((DRPCRequest)that);
+    return false;
+  }
+
+  public boolean equals(DRPCRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_func_args = true && this.is_set_func_args();
+    boolean that_present_func_args = true && that.is_set_func_args();
+    if (this_present_func_args || that_present_func_args) {
+      if (!(this_present_func_args && that_present_func_args))
+        return false;
+      if (!this.func_args.equals(that.func_args))
+        return false;
+    }
+
+    boolean this_present_request_id = true && this.is_set_request_id();
+    boolean that_present_request_id = true && that.is_set_request_id();
+    if (this_present_request_id || that_present_request_id) {
+      if (!(this_present_request_id && that_present_request_id))
+        return false;
+      if (!this.request_id.equals(that.request_id))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_func_args = true && (is_set_func_args());
+    list.add(present_func_args);
+    if (present_func_args)
+      list.add(func_args);
+
+    boolean present_request_id = true && (is_set_request_id());
+    list.add(present_request_id);
+    if (present_request_id)
+      list.add(request_id);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DRPCRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_func_args()).compareTo(other.is_set_func_args());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_func_args()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.func_args, other.func_args);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_request_id()).compareTo(other.is_set_request_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_request_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request_id, other.request_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("DRPCRequest(");
+    boolean first = true;
+
+    sb.append("func_args:");
+    if (this.func_args == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.func_args);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("request_id:");
+    if (this.request_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.request_id);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_func_args()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'func_args' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_request_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'request_id' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class DRPCRequestStandardSchemeFactory implements SchemeFactory {
+    public DRPCRequestStandardScheme getScheme() {
+      return new DRPCRequestStandardScheme();
+    }
+  }
+
+  private static class DRPCRequestStandardScheme extends StandardScheme<DRPCRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DRPCRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // FUNC_ARGS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.func_args = iprot.readString();
+              struct.set_func_args_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // REQUEST_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.request_id = iprot.readString();
+              struct.set_request_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, DRPCRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.func_args != null) {
+        oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC);
+        oprot.writeString(struct.func_args);
+        oprot.writeFieldEnd();
+      }
+      if (struct.request_id != null) {
+        oprot.writeFieldBegin(REQUEST_ID_FIELD_DESC);
+        oprot.writeString(struct.request_id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DRPCRequestTupleSchemeFactory implements SchemeFactory {
+    public DRPCRequestTupleScheme getScheme() {
+      return new DRPCRequestTupleScheme();
+    }
+  }
+
+  private static class DRPCRequestTupleScheme extends TupleScheme<DRPCRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DRPCRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.func_args);
+      oprot.writeString(struct.request_id);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DRPCRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.func_args = iprot.readString();
+      struct.set_func_args_isSet(true);
+      struct.request_id = iprot.readString();
+      struct.set_request_id_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/DebugOptions.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/DebugOptions.java b/storm-client/src/jvm/org/apache/storm/generated/DebugOptions.java
new file mode 100644
index 0000000..1ceb853
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/DebugOptions.java
@@ -0,0 +1,506 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class DebugOptions implements org.apache.thrift.TBase<DebugOptions, DebugOptions._Fields>, java.io.Serializable, Cloneable, Comparable<DebugOptions> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DebugOptions");
+
+  private static final org.apache.thrift.protocol.TField ENABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("enable", org.apache.thrift.protocol.TType.BOOL, (short)1);
+  private static final org.apache.thrift.protocol.TField SAMPLINGPCT_FIELD_DESC = new org.apache.thrift.protocol.TField("samplingpct", org.apache.thrift.protocol.TType.DOUBLE, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DebugOptionsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DebugOptionsTupleSchemeFactory());
+  }
+
+  private boolean enable; // optional
+  private double samplingpct; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ENABLE((short)1, "enable"),
+    SAMPLINGPCT((short)2, "samplingpct");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ENABLE
+          return ENABLE;
+        case 2: // SAMPLINGPCT
+          return SAMPLINGPCT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ENABLE_ISSET_ID = 0;
+  private static final int __SAMPLINGPCT_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ENABLE,_Fields.SAMPLINGPCT};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ENABLE, new org.apache.thrift.meta_data.FieldMetaData("enable", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.SAMPLINGPCT, new org.apache.thrift.meta_data.FieldMetaData("samplingpct", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DebugOptions.class, metaDataMap);
+  }
+
+  public DebugOptions() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DebugOptions(DebugOptions other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.enable = other.enable;
+    this.samplingpct = other.samplingpct;
+  }
+
+  public DebugOptions deepCopy() {
+    return new DebugOptions(this);
+  }
+
+  @Override
+  public void clear() {
+    set_enable_isSet(false);
+    this.enable = false;
+    set_samplingpct_isSet(false);
+    this.samplingpct = 0.0;
+  }
+
+  public boolean is_enable() {
+    return this.enable;
+  }
+
+  public void set_enable(boolean enable) {
+    this.enable = enable;
+    set_enable_isSet(true);
+  }
+
+  public void unset_enable() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ENABLE_ISSET_ID);
+  }
+
+  /** Returns true if field enable is set (has been assigned a value) and false otherwise */
+  public boolean is_set_enable() {
+    return EncodingUtils.testBit(__isset_bitfield, __ENABLE_ISSET_ID);
+  }
+
+  public void set_enable_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ENABLE_ISSET_ID, value);
+  }
+
+  public double get_samplingpct() {
+    return this.samplingpct;
+  }
+
+  public void set_samplingpct(double samplingpct) {
+    this.samplingpct = samplingpct;
+    set_samplingpct_isSet(true);
+  }
+
+  public void unset_samplingpct() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SAMPLINGPCT_ISSET_ID);
+  }
+
+  /** Returns true if field samplingpct is set (has been assigned a value) and false otherwise */
+  public boolean is_set_samplingpct() {
+    return EncodingUtils.testBit(__isset_bitfield, __SAMPLINGPCT_ISSET_ID);
+  }
+
+  public void set_samplingpct_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SAMPLINGPCT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ENABLE:
+      if (value == null) {
+        unset_enable();
+      } else {
+        set_enable((Boolean)value);
+      }
+      break;
+
+    case SAMPLINGPCT:
+      if (value == null) {
+        unset_samplingpct();
+      } else {
+        set_samplingpct((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ENABLE:
+      return is_enable();
+
+    case SAMPLINGPCT:
+      return get_samplingpct();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ENABLE:
+      return is_set_enable();
+    case SAMPLINGPCT:
+      return is_set_samplingpct();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DebugOptions)
+      return this.equals((DebugOptions)that);
+    return false;
+  }
+
+  public boolean equals(DebugOptions that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_enable = true && this.is_set_enable();
+    boolean that_present_enable = true && that.is_set_enable();
+    if (this_present_enable || that_present_enable) {
+      if (!(this_present_enable && that_present_enable))
+        return false;
+      if (this.enable != that.enable)
+        return false;
+    }
+
+    boolean this_present_samplingpct = true && this.is_set_samplingpct();
+    boolean that_present_samplingpct = true && that.is_set_samplingpct();
+    if (this_present_samplingpct || that_present_samplingpct) {
+      if (!(this_present_samplingpct && that_present_samplingpct))
+        return false;
+      if (this.samplingpct != that.samplingpct)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_enable = true && (is_set_enable());
+    list.add(present_enable);
+    if (present_enable)
+      list.add(enable);
+
+    boolean present_samplingpct = true && (is_set_samplingpct());
+    list.add(present_samplingpct);
+    if (present_samplingpct)
+      list.add(samplingpct);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DebugOptions other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_enable()).compareTo(other.is_set_enable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_enable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enable, other.enable);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_samplingpct()).compareTo(other.is_set_samplingpct());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_samplingpct()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.samplingpct, other.samplingpct);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("DebugOptions(");
+    boolean first = true;
+
+    if (is_set_enable()) {
+      sb.append("enable:");
+      sb.append(this.enable);
+      first = false;
+    }
+    if (is_set_samplingpct()) {
+      if (!first) sb.append(", ");
+      sb.append("samplingpct:");
+      sb.append(this.samplingpct);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class DebugOptionsStandardSchemeFactory implements SchemeFactory {
+    public DebugOptionsStandardScheme getScheme() {
+      return new DebugOptionsStandardScheme();
+    }
+  }
+
+  private static class DebugOptionsStandardScheme extends StandardScheme<DebugOptions> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DebugOptions struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ENABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.enable = iprot.readBool();
+              struct.set_enable_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SAMPLINGPCT
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.samplingpct = iprot.readDouble();
+              struct.set_samplingpct_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, DebugOptions struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_enable()) {
+        oprot.writeFieldBegin(ENABLE_FIELD_DESC);
+        oprot.writeBool(struct.enable);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_samplingpct()) {
+        oprot.writeFieldBegin(SAMPLINGPCT_FIELD_DESC);
+        oprot.writeDouble(struct.samplingpct);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DebugOptionsTupleSchemeFactory implements SchemeFactory {
+    public DebugOptionsTupleScheme getScheme() {
+      return new DebugOptionsTupleScheme();
+    }
+  }
+
+  private static class DebugOptionsTupleScheme extends TupleScheme<DebugOptions> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DebugOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_enable()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_samplingpct()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_enable()) {
+        oprot.writeBool(struct.enable);
+      }
+      if (struct.is_set_samplingpct()) {
+        oprot.writeDouble(struct.samplingpct);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DebugOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.enable = iprot.readBool();
+        struct.set_enable_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.samplingpct = iprot.readDouble();
+        struct.set_samplingpct_isSet(true);
+      }
+    }
+  }
+
+}
+


[45/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java
new file mode 100644
index 0000000..5b3866d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java
@@ -0,0 +1,409 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.AccessControl;
+import org.apache.storm.generated.AccessControlType;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.SettableBlobMeta;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IPrincipalToLocal;
+import org.apache.storm.security.auth.NimbusPrincipal;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Provides common handling of acls for Blobstores.
+ * Also contains some static utility functions related to Blobstores.
+ */
+public class BlobStoreAclHandler {
+    public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
+    private final IPrincipalToLocal _ptol;
+
+    public static final int READ = 0x01;
+    public static final int WRITE = 0x02;
+    public static final int ADMIN = 0x04;
+    public static final List<AccessControl> WORLD_EVERYTHING =
+            Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
+    public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
+    private Set<String> _supervisors;
+    private Set<String> _admins;
+    private boolean doAclValidation;
+
+    public BlobStoreAclHandler(Map conf) {
+        _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
+        _supervisors = new HashSet<String>();
+        _admins = new HashSet<String>();
+        if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
+            _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
+        }
+        if (conf.containsKey(Config.NIMBUS_ADMINS)) {
+            _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
+        }
+        if (conf.containsKey(Config.STORM_BLOBSTORE_ACL_VALIDATION_ENABLED)) {
+           doAclValidation = (boolean)conf.get(Config.STORM_BLOBSTORE_ACL_VALIDATION_ENABLED);
+        }
+    }
+
+    private static AccessControlType parseACLType(String type) {
+        if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
+            return AccessControlType.OTHER;
+        } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
+            return AccessControlType.USER;
+        }
+        throw new IllegalArgumentException(type+" is not a valid access control type");
+    }
+
+    private static int parseAccess(String access) {
+        int ret = 0;
+        for (char c: access.toCharArray()) {
+            if ('r' == c) {
+                ret = ret | READ;
+            } else if ('w' == c) {
+                ret = ret | WRITE;
+            } else if ('a' == c) {
+                ret = ret | ADMIN;
+            } else if ('-' == c) {
+                //ignored
+            } else {
+                throw new IllegalArgumentException("");
+            }
+        }
+        return ret;
+    }
+
+    public static AccessControl parseAccessControl(String str) {
+        String[] parts = str.split(":");
+        String type = "other";
+        String name = "";
+        String access = "-";
+        if (parts.length > 3) {
+            throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
+        } else if (parts.length == 1) {
+            type = "other";
+            name = "";
+            access = parts[0];
+        } else if (parts.length == 2) {
+            type = "user";
+            name = parts[0];
+            access = parts[1];
+        } else if (parts.length == 3) {
+            type = parts[0];
+            name = parts[1];
+            access = parts[2];
+        }
+        AccessControl ret = new AccessControl();
+        ret.set_type(parseACLType(type));
+        ret.set_name(name);
+        ret.set_access(parseAccess(access));
+        return ret;
+    }
+
+    private static String accessToString(int access) {
+        StringBuilder ret = new StringBuilder();
+        ret.append(((access & READ) > 0) ? "r" : "-");
+        ret.append(((access & WRITE) > 0) ? "w" : "-");
+        ret.append(((access & ADMIN) > 0) ? "a" : "-");
+        return ret.toString();
+    }
+
+    public static String accessControlToString(AccessControl ac) {
+        StringBuilder ret = new StringBuilder();
+        switch(ac.get_type()) {
+            case OTHER:
+                ret.append("o");
+                break;
+            case USER:
+                ret.append("u");
+                break;
+            default:
+                throw new IllegalArgumentException("Don't know what a type of "+ac.get_type()+" means ");
+        }
+        ret.append(":");
+        if (ac.is_set_name()) {
+            ret.append(ac.get_name());
+        }
+        ret.append(":");
+        ret.append(accessToString(ac.get_access()));
+        return ret.toString();
+    }
+
+    public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
+        Set<String> aclUsers = new HashSet<>();
+        List<String> duplicateUsers = new ArrayList<>();
+        for (AccessControl acl : acls) {
+            String aclUser = acl.get_name();
+            if (!StringUtils.isEmpty(aclUser) && !aclUsers.add(aclUser)) {
+                LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
+                duplicateUsers.add(aclUser);
+            }
+        }
+        if (duplicateUsers.size() > 0) {
+            String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
+                    + " can't appear more than once in the ACLs for key [" + key +"].";
+            throw new AuthorizationException(errorMessage);
+        }
+    }
+
+    private Set<String> constructUserFromPrincipals(Subject who) {
+        Set<String> user = new HashSet<String>();
+        if (who != null) {
+            for (Principal p : who.getPrincipals()) {
+                user.add(_ptol.toLocal(p));
+            }
+        }
+        return user;
+    }
+
+    private boolean isAdmin(Subject who) {
+        Set<String> user = constructUserFromPrincipals(who);
+        for (String u : user) {
+            if (_admins.contains(u)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private boolean isReadOperation(int operation) {
+        if (operation == 1) {
+            return true;
+        }
+        return false;
+    }
+
+    private boolean isSupervisor(Subject who, int operation) {
+        Set<String> user = constructUserFromPrincipals(who);
+        if (isReadOperation(operation)) {
+            for (String u : user) {
+                if (_supervisors.contains(u)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    private boolean isNimbus(Subject who) {
+        Set<Principal> principals;
+        boolean isNimbusInstance = false;
+        if (who != null) {
+            principals = who.getPrincipals();
+            for (Principal principal : principals) {
+                if (principal instanceof NimbusPrincipal) {
+                    isNimbusInstance = true;
+                }
+            }
+        }
+        return isNimbusInstance;
+    }
+
+    public boolean checkForValidUsers(Subject who, int mask) {
+        return isNimbus(who) || isAdmin(who) || isSupervisor(who,mask);
+    }
+
+    /**
+     * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
+     */
+    public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key) throws AuthorizationException {
+        hasAnyPermissions(acl, (READ|WRITE|ADMIN), who, key);
+    }
+
+    /**
+     * Validates if the user has any of the permissions
+     * mentioned in the mask.
+     * @param acl ACL for the key.
+     * @param mask mask holds the cumulative value of
+     * READ = 1, WRITE = 2 or ADMIN = 4 permissions.
+     * mask = 1 implies READ privilege.
+     * mask = 5 implies READ and ADMIN privileges.
+     * @param who Is the user against whom the permissions
+     * are validated for a key using the ACL and the mask.
+     * @param key Key used to identify the blob.
+     * @throws AuthorizationException
+     */
+    public void hasAnyPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
+        if (!doAclValidation) {
+            return;
+        }
+        Set<String> user = constructUserFromPrincipals(who);
+        LOG.debug("user {}", user);
+        if (checkForValidUsers(who, mask)) {
+            return;
+        }
+        for (AccessControl ac : acl) {
+            int allowed = getAllowed(ac, user);
+            LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
+            if ((allowed & mask) > 0) {
+                return;
+            }
+        }
+        throw new AuthorizationException(
+                user + " does not have access to " + key);
+    }
+
+    /**
+     * Validates if the user has at least the set of permissions
+     * mentioned in the mask.
+     * @param acl ACL for the key.
+     * @param mask mask holds the cumulative value of
+     * READ = 1, WRITE = 2 or ADMIN = 4 permissions.
+     * mask = 1 implies READ privilege.
+     * mask = 5 implies READ and ADMIN privileges.
+     * @param who Is the user against whom the permissions
+     * are validated for a key using the ACL and the mask.
+     * @param key Key used to identify the blob.
+     * @throws AuthorizationException
+     */
+    public void hasPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
+        if (!doAclValidation) {
+            return;
+        }
+        Set<String> user = constructUserFromPrincipals(who);
+        LOG.debug("user {}", user);
+        if (checkForValidUsers(who, mask)) {
+            return;
+        }
+        for (AccessControl ac : acl) {
+            int allowed = getAllowed(ac, user);
+            mask = ~allowed & mask;
+            LOG.debug(" user: {} allowed: {} disallowed: {} key: {}", user, allowed, mask, key);
+        }
+        if (mask == 0) {
+            return;
+        }
+        throw new AuthorizationException(
+                user + " does not have " + namedPerms(mask) + " access to " + key);
+    }
+
+    public void normalizeSettableBlobMeta(String key, SettableBlobMeta meta, Subject who, int opMask) {
+        meta.set_acl(normalizeSettableACLs(key, meta.get_acl(), who, opMask));
+    }
+
+    private String namedPerms(int mask) {
+        StringBuilder b = new StringBuilder();
+        b.append("[");
+        if ((mask & READ) > 0) {
+            b.append("READ ");
+        }
+        if ((mask & WRITE) > 0) {
+            b.append("WRITE ");
+        }
+        if ((mask & ADMIN) > 0) {
+            b.append("ADMIN ");
+        }
+        b.append("]");
+        return b.toString();
+    }
+
+    private int getAllowed(AccessControl ac, Set<String> users) {
+        switch (ac.get_type()) {
+            case OTHER:
+                return ac.get_access();
+            case USER:
+                if (users.contains(ac.get_name())) {
+                    return ac.get_access();
+                }
+                return 0;
+            default:
+                return 0;
+        }
+    }
+
+    private List<AccessControl> removeBadACLs(List<AccessControl> accessControls) {
+        List<AccessControl> resultAcl = new ArrayList<AccessControl>();
+        for (AccessControl control : accessControls) {
+            if(control.get_type().equals(AccessControlType.OTHER) && (control.get_access() == 0 )) {
+                LOG.debug("Removing invalid blobstore world ACL " +
+                        BlobStoreAclHandler.accessControlToString(control));
+                continue;
+            }
+            resultAcl.add(control);
+        }
+        return resultAcl;
+    }
+
+    private final List<AccessControl> normalizeSettableACLs(String key, List<AccessControl> acls, Subject who,
+                                                            int opMask) {
+        List<AccessControl> cleanAcls = removeBadACLs(acls);
+        Set<String> userNames = getUserNamesFromSubject(who);
+        for (String user : userNames) {
+            fixACLsForUser(cleanAcls, user, opMask);
+        }
+        if ((who == null || userNames.isEmpty()) && !worldEverything(acls)) {
+            cleanAcls.addAll(BlobStoreAclHandler.WORLD_EVERYTHING);
+            LOG.debug("Access Control for key {} is normalized to world everything {}", key, cleanAcls);
+            if (!acls.isEmpty())
+                LOG.warn("Access control for blob with key {} is normalized to WORLD_EVERYTHING", key);
+        }
+        return cleanAcls;
+    }
+
+    private boolean worldEverything(List<AccessControl> acls) {
+        boolean isWorldEverything = false;
+        for (AccessControl acl : acls) {
+            if (acl.get_type() == AccessControlType.OTHER && acl.get_access() == (READ|WRITE|ADMIN)) {
+                isWorldEverything = true;
+                break;
+            }
+        }
+        return isWorldEverything;
+    }
+
+    private void fixACLsForUser(List<AccessControl> acls, String user, int mask) {
+        boolean foundUserACL = false;
+        for (AccessControl control : acls) {
+            if (control.get_type() == AccessControlType.USER && control.get_name().equals(user)) {
+                int currentAccess = control.get_access();
+                if ((currentAccess & mask) != mask) {
+                    control.set_access(currentAccess | mask);
+                }
+                foundUserACL = true;
+                break;
+            }
+        }
+        if (!foundUserACL) {
+            AccessControl userACL = new AccessControl();
+            userACL.set_type(AccessControlType.USER);
+            userACL.set_name(user);
+            userACL.set_access(mask);
+            acls.add(userACL);
+        }
+    }
+
+    private Set<String> getUserNamesFromSubject(Subject who) {
+        Set<String> user = new HashSet<String>();
+        if (who != null) {
+            for(Principal p: who.getPrincipals()) {
+                user.add(_ptol.toLocal(p));
+            }
+        }
+        return user;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreFile.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreFile.java b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreFile.java
new file mode 100644
index 0000000..9de2f4a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreFile.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+import org.apache.storm.generated.SettableBlobMeta;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.regex.Pattern;
+
+/**
+ * Provides an base implementation for creating a blobstore based on file backed storage.
+ */
+public abstract class BlobStoreFile {
+    public static final Logger LOG = LoggerFactory.getLogger(BlobStoreFile.class);
+
+    protected static final String TMP_EXT = ".tmp";
+    protected static final Pattern TMP_NAME_PATTERN = Pattern.compile("^\\d+\\" + TMP_EXT + "$");
+    protected static final String BLOBSTORE_DATA_FILE = "data";
+
+    public abstract void delete() throws IOException;
+    public abstract String getKey();
+    public abstract boolean isTmp();
+    public abstract void setMetadata(SettableBlobMeta meta);
+    public abstract SettableBlobMeta getMetadata();
+    public abstract long getModTime() throws IOException;
+    public abstract InputStream getInputStream() throws IOException;
+    public abstract OutputStream getOutputStream() throws IOException;
+    public abstract void commit() throws IOException;
+    public abstract void cancel() throws IOException;
+    public abstract long getFileLength() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java
new file mode 100644
index 0000000..a1499aa
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java
@@ -0,0 +1,201 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.ReadableBlobMeta;
+import org.apache.storm.generated.SettableBlobMeta;
+import org.apache.storm.generated.KeyAlreadyExistsException;
+import org.apache.storm.generated.KeyNotFoundException;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.NimbusClient;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * The ClientBlobStore has two concrete implementations
+ * 1. NimbusBlobStore
+ * 2. HdfsClientBlobStore
+ *
+ * Create, update, read and delete are some of the basic operations defined by this interface.
+ * Each operation is validated for permissions against an user. We currently have NIMBUS_ADMINS and SUPERVISOR_ADMINS
+ * configuration. NIMBUS_ADMINS are given READ, WRITE and ADMIN access whereas the SUPERVISOR_ADMINS are given READ
+ * access in order to read and download the blobs form the nimbus.
+ *
+ * The ACLs for the blob store are validated against whether the subject is a NIMBUS_ADMIN, SUPERVISOR_ADMIN or USER
+ * who has read, write or admin privileges in order to perform respective operations on the blob.
+ *
+ * For more detailed implementation
+ * @see org.apache.storm.blobstore.NimbusBlobStore
+ * @see org.apache.storm.blobstore.LocalFsBlobStore
+ * @see org.apache.storm.hdfs.blobstore.HdfsClientBlobStore
+ * @see org.apache.storm.hdfs.blobstore.HdfsBlobStore
+ */
+public abstract class ClientBlobStore implements Shutdownable {
+    protected Map conf;
+
+    public interface WithBlobstore {
+        void run(ClientBlobStore blobStore) throws Exception;
+    }
+
+    public static void withConfiguredClient(WithBlobstore withBlobstore) throws Exception {
+        Map<String, Object> conf = ConfigUtils.readStormConfig();
+        ClientBlobStore blobStore = Utils.getClientBlobStore(conf);
+
+        try {
+            withBlobstore.run(blobStore);
+        } finally {
+            blobStore.shutdown();
+        }
+    }
+
+    /**
+     * Sets up the client API by parsing the configs.
+     * @param conf The storm conf containing the config details.
+     */
+    public abstract void prepare(Map conf);
+
+    /**
+     * Client facing API to create a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyAlreadyExistsException
+     */
+    protected abstract AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException;
+
+    /**
+     * Client facing API to update a blob.
+     * @param key blob key name.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract AtomicOutputStream updateBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to read the metadata information.
+     * @param key blob key name.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to set the metadata for a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    protected abstract void setBlobMetaToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to delete a blob.
+     * @param key blob key name.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to read a blob.
+     * @param key blob key name.
+     * @return an InputStream to read the metadata for a blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * @return Iterator for a list of keys currently present in the blob store.
+     */
+    public abstract Iterator<String> listKeys();
+
+    /**
+     * Client facing API to read the replication of a blob.
+     * @param key blob key name.
+     * @return int indicates the replication factor of a blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to update the replication of a blob.
+     * @param key blob key name.
+     * @param replication int indicates the replication factor a blob has to be set.
+     * @return int indicates the replication factor of a blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to set a nimbus client.
+     * @param conf storm conf
+     * @param client NimbusClient
+     * @return indicates where the client connection has been setup.
+     */
+    public abstract boolean setClient(Map conf, NimbusClient client);
+
+    /**
+     * Creates state inside a zookeeper.
+     * Required for blobstore to write to zookeeper
+     * when Nimbus HA is turned on in order to maintain
+     * state consistency
+     * @param key
+     */
+    public abstract void createStateInZookeeper(String key);
+
+    /**
+     * Client facing API to create a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyAlreadyExistsException
+     */
+    public final AtomicOutputStream createBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException {
+        if (meta !=null && meta.is_set_acl()) {
+            BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
+        }
+        return createBlobToExtend(key, meta);
+    }
+
+    /**
+     * Client facing API to set the metadata for a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public final void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException {
+        if (meta !=null && meta.is_set_acl()) {
+            BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
+        }
+        setBlobMetaToExtend(key, meta);
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/InputStreamWithMeta.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/InputStreamWithMeta.java b/storm-client/src/jvm/org/apache/storm/blobstore/InputStreamWithMeta.java
new file mode 100644
index 0000000..6600a00
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/InputStreamWithMeta.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public abstract class InputStreamWithMeta extends InputStream {
+    public abstract long getVersion() throws IOException;
+    public abstract long getFileLength() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/KeyFilter.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/KeyFilter.java b/storm-client/src/jvm/org/apache/storm/blobstore/KeyFilter.java
new file mode 100644
index 0000000..c2d69e1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/KeyFilter.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+public interface KeyFilter<R> {
+    R filter(String key);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java
new file mode 100644
index 0000000..5b7713d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java
@@ -0,0 +1,428 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.blobstore;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.BeginDownloadResult;
+import org.apache.storm.generated.ListBlobsResult;
+import org.apache.storm.generated.ReadableBlobMeta;
+import org.apache.storm.generated.SettableBlobMeta;
+import org.apache.storm.generated.KeyAlreadyExistsException;
+import org.apache.storm.generated.KeyNotFoundException;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * NimbusBlobStore is a USER facing client API to perform
+ * basic operations such as create, update, delete and read
+ * for local and hdfs blob store.
+ *
+ * For local blob store it is also the client facing API for
+ * supervisor in order to download blobs from nimbus.
+ */
+public class NimbusBlobStore extends ClientBlobStore implements AutoCloseable {
+    private static final Logger LOG = LoggerFactory.getLogger(NimbusBlobStore.class);
+
+    public class NimbusKeyIterator implements Iterator<String> {
+        private ListBlobsResult listBlobs = null;
+        private int offset = 0;
+        private boolean eof = false;
+
+        public NimbusKeyIterator(ListBlobsResult listBlobs) {
+            this.listBlobs = listBlobs;
+            this.eof = (listBlobs.get_keys_size() == 0);
+        }
+
+        private boolean isCacheEmpty() {
+            return listBlobs.get_keys_size() <= offset;
+        }
+
+        private void readMore() throws TException {
+            if (!eof) {
+                offset = 0;
+                synchronized(client) {
+                    listBlobs = client.getClient().listBlobs(listBlobs.get_session());
+                }
+                if (listBlobs.get_keys_size() == 0) {
+                    eof = true;
+                }
+            }
+        }
+
+        @Override
+        public synchronized boolean hasNext() {
+            try {
+                if (isCacheEmpty()) {
+                    readMore();
+                }
+            } catch (TException e) {
+                throw new RuntimeException(e);
+            }
+            return !eof;
+        }
+
+        @Override
+        public synchronized String next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            String ret = listBlobs.get_keys().get(offset);
+            offset++;
+            return ret;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException("Delete Not Supported");
+        }
+    }
+
+    public class NimbusDownloadInputStream extends InputStreamWithMeta {
+        private BeginDownloadResult beginBlobDownload;
+        private byte[] buffer = null;
+        private int offset = 0;
+        private int end = 0;
+        private boolean eof = false;
+
+        public NimbusDownloadInputStream(BeginDownloadResult beginBlobDownload) {
+            this.beginBlobDownload = beginBlobDownload;
+        }
+
+        @Override
+        public long getVersion() throws IOException {
+            return beginBlobDownload.get_version();
+        }
+
+        @Override
+        public synchronized int read() throws IOException {
+            try {
+                if (isEmpty()) {
+                    readMore();
+                    if (eof) {
+                        return -1;
+                    }
+                }
+                int length = Math.min(1, available());
+                if (length == 0) {
+                    return -1;
+                }
+                int ret = buffer[offset];
+                offset += length;
+                return ret;
+            } catch(TException exp) {
+                throw new IOException(exp);
+            }
+        }
+
+        @Override
+        public synchronized int read(byte[] b, int off, int len) throws IOException {
+            try {
+                if (isEmpty()) {
+                    readMore();
+                    if (eof) {
+                        return -1;
+                    }
+                }
+                int length = Math.min(len, available());
+                System.arraycopy(buffer, offset, b, off, length);
+                offset += length;
+                return length;
+            } catch(TException exp) {
+                throw new IOException(exp);
+            }
+        }
+
+        private boolean isEmpty() {
+            return buffer == null || offset >= end;
+        }
+
+        private void readMore() throws TException {
+            if (!eof) {
+                ByteBuffer buff;
+                synchronized(client) {
+                    buff = client.getClient().downloadBlobChunk(beginBlobDownload.get_session());
+                }
+                buffer = buff.array();
+                offset = buff.arrayOffset() + buff.position();
+                int length = buff.remaining();
+                end = offset + length;
+                if (length == 0) {
+                    eof = true;
+                }
+            }
+        }
+
+        @Override
+        public synchronized int read(byte[] b) throws IOException {
+            return read(b, 0, b.length);
+        }
+
+        @Override
+        public synchronized int available() {
+            return buffer == null ? 0 : (end - offset);
+        }
+
+        @Override
+        public long getFileLength() {
+            return beginBlobDownload.get_data_size();
+        }
+    }
+
+    public class NimbusUploadAtomicOutputStream extends AtomicOutputStream {
+        private String session;
+        private int maxChunkSize = 4096;
+        private String key;
+
+        public NimbusUploadAtomicOutputStream(String session, int bufferSize, String key) {
+            this.session = session;
+            this.maxChunkSize = bufferSize;
+            this.key = key;
+        }
+
+        @Override
+        public void cancel() throws IOException {
+            try {
+                synchronized(client) {
+                    client.getClient().cancelBlobUpload(session);
+                }
+            } catch (TException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            try {
+                synchronized(client) {
+                    client.getClient().uploadBlobChunk(session, ByteBuffer.wrap(new byte[] {(byte)b}));
+                }
+            } catch (TException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        @Override
+        public void write(byte []b) throws IOException {
+            write(b, 0, b.length);
+        }
+
+        @Override
+        public void write(byte []b, int offset, int len) throws IOException {
+            try {
+                int end = offset + len;
+                for (int realOffset = offset; realOffset < end; realOffset += maxChunkSize) {
+                    int realLen = Math.min(end - realOffset, maxChunkSize);
+                    LOG.debug("Writing {} bytes of {} remaining",realLen,(end-realOffset));
+                    synchronized(client) {
+                        client.getClient().uploadBlobChunk(session, ByteBuffer.wrap(b, realOffset, realLen));
+                    }
+                }
+            } catch (TException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        @Override
+        public void close() throws IOException {
+            try {
+                synchronized(client) {
+                    client.getClient().finishBlobUpload(session);
+                    client.getClient().createStateInZookeeper(key);
+                }
+            } catch (TException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    private NimbusClient client;
+    private int bufferSize = 4096;
+
+    @Override
+    public void prepare(Map conf) {
+        this.client = NimbusClient.getConfiguredClient(conf);
+        if (conf != null) {
+            this.bufferSize = ObjectReader.getInt(conf.get(Config.STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES), bufferSize);
+        }
+    }
+
+    @Override
+    protected AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta)
+            throws AuthorizationException, KeyAlreadyExistsException {
+        try {
+            synchronized(client) {
+                return new NimbusUploadAtomicOutputStream(client.getClient().beginCreateBlob(key, meta), this.bufferSize, key);
+            }
+        } catch (AuthorizationException | KeyAlreadyExistsException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public AtomicOutputStream updateBlob(String key)
+            throws AuthorizationException, KeyNotFoundException {
+        try {
+            synchronized(client) {
+                return new NimbusUploadAtomicOutputStream(client.getClient().beginUpdateBlob(key), this.bufferSize, key);
+            }
+        } catch (AuthorizationException | KeyNotFoundException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException {
+        try {
+            synchronized(client) {
+                return client.getClient().getBlobMeta(key);
+            }
+        } catch (AuthorizationException | KeyNotFoundException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    protected void setBlobMetaToExtend(String key, SettableBlobMeta meta)
+            throws AuthorizationException, KeyNotFoundException {
+        try {
+            synchronized(client) {
+                client.getClient().setBlobMeta(key, meta);
+            }
+        } catch (AuthorizationException | KeyNotFoundException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException {
+        try {
+            synchronized(client) {
+                client.getClient().deleteBlob(key);
+            }
+        } catch (AuthorizationException | KeyNotFoundException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void createStateInZookeeper(String key) {
+        try {
+            synchronized(client) {
+                client.getClient().createStateInZookeeper(key);
+            }
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException {
+        try {
+            synchronized(client) {
+                return new NimbusDownloadInputStream(client.getClient().beginBlobDownload(key));
+            }
+        } catch (AuthorizationException | KeyNotFoundException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public Iterator<String> listKeys() {
+        try {
+            synchronized(client) {
+                return new NimbusKeyIterator(client.getClient().listBlobs(""));
+            }
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException {
+        try {
+            return client.getClient().getBlobReplication(key);
+        } catch (AuthorizationException | KeyNotFoundException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException {
+        try {
+            return client.getClient().updateBlobReplication(key, replication);
+        } catch (AuthorizationException | KeyNotFoundException exp) {
+            throw exp;
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public boolean setClient(Map conf, NimbusClient client) {
+        if (this.client != null) {
+            this.client.close();
+        }
+        this.client = client;
+        if (conf != null) {
+            this.bufferSize = ObjectReader.getInt(conf.get(Config.STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES), bufferSize);
+        }
+        return true;
+    }
+
+    @Override
+    protected void finalize() {
+        shutdown();
+    }
+
+    @Override
+    public void shutdown() {
+        if (client != null) {
+            client.close();
+            client = null;
+        }
+    }
+
+    @Override
+    public void close() {
+        shutdown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java b/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java
new file mode 100644
index 0000000..007a958
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java
@@ -0,0 +1,595 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.bolt;
+
+
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.windowing.TimestampExtractor;
+import org.apache.storm.windowing.TupleWindow;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+
+public class JoinBolt extends BaseWindowedBolt {
+
+    private OutputCollector collector;
+
+    // Map[StreamName -> Map[Key -> List<Tuple>]  ]
+    HashMap<String, HashMap<Object, ArrayList<Tuple> >> hashedInputs = new HashMap<>(); // holds remaining streams
+
+    // Map[StreamName -> JoinInfo]
+    protected LinkedHashMap<String, JoinInfo> joinCriteria = new LinkedHashMap<>();
+    protected FieldSelector[] outputFields;  // specified via bolt.select() ... used in declaring Output fields
+//    protected String[] dotSeparatedOutputFieldNames; // fieldNames in x.y.z format w/o stream name, used for naming output fields
+    protected String outputStreamName;
+
+    // Use streamId, source component name OR field in tuple to distinguish incoming tuple streams
+    public enum Selector { STREAM, SOURCE }
+    protected final Selector selectorType;
+
+
+    /**
+     * Calls  JoinBolt(Selector.SOURCE, sourceId, fieldName)
+     * @param sourceId   Id of source component (spout/bolt) from which this bolt is receiving data
+     * @param fieldName  the field to use for joining the stream (x.y.z format)
+     */
+    public JoinBolt(String sourceId, String fieldName) {
+        this(Selector.SOURCE, sourceId, fieldName);
+    }
+    /**
+     *
+     * Introduces the first stream to start the join with. Equivalent SQL ...
+     *       select .... from srcOrStreamId ...
+     * @param type Specifies whether 'srcOrStreamId' refers to stream name/source component
+     * @param srcOrStreamId name of stream OR source component
+     * @param fieldName the field to use for joining the stream (x.y.z format)
+     */
+    public JoinBolt(Selector type, String srcOrStreamId, String fieldName) {
+        selectorType = type;
+
+        joinCriteria.put(srcOrStreamId, new JoinInfo(  new FieldSelector( srcOrStreamId, fieldName) ) );
+    }
+
+    /**
+     * Optional. Allows naming the output stream of this bolt. If not specified, the emits will happen on
+     * 'default' stream.
+     */
+    public JoinBolt withOutputStream(String streamName) {
+        this.outputStreamName = streamName;
+        return this;
+    }
+
+    /**
+     * Performs inner Join with the newStream.
+     *  SQL    :   from priorStream inner join newStream on newStream.field = priorStream.field1
+     *  same as:   new WindowedQueryBolt(priorStream,field1). join(newStream, field, priorStream);
+     *
+     *  Note: priorStream must be previously joined.
+     *    Valid ex:    new WindowedQueryBolt(s1,k1). join(s2,k2, s1). join(s3,k3, s2);
+     *    Invalid ex:  new WindowedQueryBolt(s1,k1). join(s3,k3, s2). join(s2,k2, s1);
+     *
+     *    @param newStream  Either stream name or name of upstream component
+     *    @param field the field on which to perform the join
+     */
+    public JoinBolt join(String newStream, String field, String priorStream) {
+        return joinCommon(newStream, field, priorStream, JoinType.INNER);
+    }
+
+    /**
+     * Performs left Join with the newStream.
+     *  SQL    :   from stream1  left join stream2  on stream2.field = stream1.field1
+     *  same as:   new  WindowedQueryBolt(stream1, field1). leftJoin(stream2, field, stream1);
+     *
+     *  Note: priorStream must be previously joined
+     *    Valid ex:    new WindowedQueryBolt(s1,k1). leftJoin(s2,k2, s1). leftJoin(s3,k3, s2);
+     *    Invalid ex:  new WindowedQueryBolt(s1,k1). leftJoin(s3,k3, s2). leftJoin(s2,k2, s1);
+     *
+     *    @param newStream  Either a name of a stream or an upstream component
+     *    @param field the field on which to perform the join
+     */
+    public JoinBolt leftJoin(String newStream, String field, String priorStream) {
+        return joinCommon(newStream, field, priorStream, JoinType.LEFT);
+    }
+
+    private JoinBolt joinCommon(String newStream, String fieldDescriptor, String priorStream, JoinType joinType) {
+        if (hashedInputs.containsKey(newStream)) {
+            throw new IllegalArgumentException("'" + newStream + "' is already part of join. Cannot join with it more than once.");
+        }
+        hashedInputs.put(newStream, new HashMap<Object, ArrayList<Tuple>>());
+        JoinInfo joinInfo = joinCriteria.get(priorStream);
+        if( joinInfo==null )
+            throw new IllegalArgumentException("Stream '" + priorStream + "' was not previously declared");
+
+        FieldSelector field = new FieldSelector(newStream, fieldDescriptor);
+        joinCriteria.put(newStream, new JoinInfo(field, priorStream, joinInfo, joinType) );
+        return this;
+    }
+
+    /**
+     * Specify projection fields. i.e. Specifies the fields to include in the output.
+     *      e.g: .select("field1, stream2:field2, field3")
+     * Nested Key names are supported for nested types:
+     *      e.g: .select("outerKey1.innerKey1, outerKey1.innerKey2, stream3:outerKey2.innerKey3)"
+     * Inner types (non leaf) must be Map<> in order to support nested lookup using this dot notation
+     * This selected fields implicitly declare the output fieldNames for the bolt based.
+     * @param commaSeparatedKeys
+     * @return
+     */
+    public JoinBolt select(String commaSeparatedKeys) {
+        String[] fieldNames = commaSeparatedKeys.split(",");
+
+        outputFields = new FieldSelector[fieldNames.length];
+        for (int i = 0; i < fieldNames.length; i++) {
+            outputFields[i] = new FieldSelector(fieldNames[i]);
+        }
+        return this;
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        String[] outputFieldNames = new String[outputFields.length];
+        for( int i=0; i<outputFields.length; ++i ) {
+            outputFieldNames[i] = outputFields[i].getOutputName() ;
+        }
+        if (outputStreamName!=null) {
+            declarer.declareStream(outputStreamName, new Fields(outputFieldNames));
+        } else {
+            declarer.declare(new Fields(outputFieldNames));
+        }
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.collector = collector;
+        // initialize the hashedInputs data structure
+        int i=0;
+        for ( String stream : joinCriteria.keySet() ) {
+            if(i>0) {
+                hashedInputs.put(stream, new HashMap<Object, ArrayList<Tuple>>());
+            }
+            ++i;
+        }
+        if(outputFields ==null) {
+            throw new IllegalArgumentException("Must specify output fields via .select() method.");
+        }
+    }
+
+    @Override
+    public void execute(TupleWindow inputWindow) {
+        // 1) Perform Join
+        List<Tuple> currentWindow = inputWindow.get();
+        JoinAccumulator joinResult = hashJoin(currentWindow);
+
+        // 2) Emit results
+        for (ResultRecord resultRecord : joinResult.getRecords()) {
+            ArrayList<Object> outputTuple = resultRecord.getOutputFields();
+            if ( outputStreamName==null )
+                // explicit anchoring emits to corresponding input tuples only, as default window anchoring will anchor them to all tuples in window
+                collector.emit( resultRecord.tupleList, outputTuple );
+            else
+                // explicitly anchor emits to corresponding input tuples only, as default window anchoring will anchor them to all tuples in window
+                collector.emit( outputStreamName, resultRecord.tupleList, outputTuple );
+        }
+    }
+
+    private void clearHashedInputs() {
+        for (HashMap<Object, ArrayList<Tuple>> mappings : hashedInputs.values()) {
+            mappings.clear();
+        }
+    }
+
+    protected JoinAccumulator hashJoin(List<Tuple> tuples) {
+        clearHashedInputs();
+
+        JoinAccumulator probe = new JoinAccumulator();
+
+        // 1) Build phase - Segregate tuples in the Window into streams.
+        //    First stream's tuples go into probe, rest into HashMaps in hashedInputs
+        String firstStream = joinCriteria.keySet().iterator().next();
+        for (Tuple tuple : tuples) {
+            String streamId = getStreamSelector(tuple);
+            if ( ! streamId.equals(firstStream) ) {
+                Object field = getJoinField(streamId, tuple);
+                ArrayList<Tuple> recs = hashedInputs.get(streamId).get(field);
+                if(recs == null) {
+                    recs = new ArrayList<Tuple>();
+                    hashedInputs.get(streamId).put(field, recs);
+                }
+                recs.add(tuple);
+
+            }  else {
+                ResultRecord probeRecord = new ResultRecord(tuple, joinCriteria.size() == 1);
+                probe.insert( probeRecord );  // first stream's data goes into the probe
+            }
+        }
+
+        // 2) Join the streams in order of streamJoinOrder
+        int i=0;
+        for (String streamName : joinCriteria.keySet() ) {
+            boolean finalJoin = (i==joinCriteria.size()-1);
+            if(i>0) {
+                probe = doJoin(probe, hashedInputs.get(streamName), joinCriteria.get(streamName), finalJoin);
+            }
+            ++i;
+        }
+
+
+        return probe;
+    }
+
+    // Dispatches to the right join method (inner/left/right/outer) based on the joinInfo.joinType
+    protected JoinAccumulator doJoin(JoinAccumulator probe, HashMap<Object, ArrayList<Tuple>> buildInput, JoinInfo joinInfo, boolean finalJoin) {
+        final JoinType joinType = joinInfo.getJoinType();
+        switch ( joinType ) {
+            case INNER:
+                return doInnerJoin(probe, buildInput, joinInfo, finalJoin);
+            case LEFT:
+                return doLeftJoin(probe, buildInput, joinInfo, finalJoin);
+            case RIGHT:
+            case OUTER:
+            default:
+                throw new RuntimeException("Unsupported join type : " + joinType.name() );
+        }
+    }
+
+    // inner join - core implementation
+    protected JoinAccumulator doInnerJoin(JoinAccumulator probe, Map<Object, ArrayList<Tuple>> buildInput, JoinInfo joinInfo, boolean finalJoin) {
+        String[] probeKeyName = joinInfo.getOtherField();
+        JoinAccumulator result = new JoinAccumulator();
+        FieldSelector fieldSelector = new FieldSelector(joinInfo.other.getStreamName(), probeKeyName);
+        for (ResultRecord rec : probe.getRecords()) {
+            Object probeKey = rec.getField(fieldSelector);
+            if (probeKey!=null) {
+                ArrayList<Tuple> matchingBuildRecs = buildInput.get(probeKey);
+                if(matchingBuildRecs!=null) {
+                    for (Tuple matchingRec : matchingBuildRecs) {
+                        ResultRecord mergedRecord = new ResultRecord(rec, matchingRec, finalJoin);
+                        result.insert(mergedRecord);
+                    }
+                }
+            }
+        }
+        return result;
+    }
+
+    // left join - core implementation
+    protected JoinAccumulator doLeftJoin(JoinAccumulator probe, Map<Object, ArrayList<Tuple>> buildInput, JoinInfo joinInfo, boolean finalJoin) {
+        String[] probeKeyName = joinInfo.getOtherField();
+        JoinAccumulator result = new JoinAccumulator();
+        FieldSelector fieldSelector = new FieldSelector(joinInfo.other.getStreamName(), probeKeyName);
+        for (ResultRecord rec : probe.getRecords()) {
+            Object probeKey = rec.getField(fieldSelector);
+            if (probeKey!=null) {
+                ArrayList<Tuple> matchingBuildRecs = buildInput.get(probeKey); // ok if its return null
+                if (matchingBuildRecs!=null && !matchingBuildRecs.isEmpty() ) {
+                    for (Tuple matchingRec : matchingBuildRecs) {
+                        ResultRecord mergedRecord = new ResultRecord(rec, matchingRec, finalJoin);
+                        result.insert(mergedRecord);
+                    }
+                } else {
+                    ResultRecord mergedRecord = new ResultRecord(rec, null, finalJoin);
+                    result.insert(mergedRecord);
+                }
+
+            }
+        }
+        return result;
+    }
+
+
+    // Identify the join field for the stream, and look it up in 'tuple'. field can be nested field:  outerKey.innerKey
+    private Object getJoinField(String streamId, Tuple tuple) {
+        JoinInfo ji = joinCriteria.get(streamId);
+        if(ji==null) {
+            throw new RuntimeException("Join information for '" + streamId + "' not found. Check the join clauses.");
+        }
+        return lookupField(ji.getJoinField(), tuple);
+    }
+
+    // Returns either the source component name or the stream name for the tuple
+    private String getStreamSelector(Tuple ti) {
+        switch (selectorType) {
+            case STREAM:
+                return ti.getSourceStreamId();
+            case SOURCE:
+                return ti.getSourceComponent();
+            default:
+                throw new RuntimeException(selectorType + " stream selector type not yet supported");
+        }
+    }
+
+
+    protected enum JoinType {INNER, LEFT, RIGHT, OUTER}
+
+    /** Describes how to join the other stream with the current stream */
+    protected static class JoinInfo implements Serializable {
+        final static long serialVersionUID = 1L;
+
+        private JoinType joinType;        // nature of join
+        private FieldSelector field;           // field for the current stream
+        private FieldSelector other;      // field for the other (2nd) stream
+
+
+        public JoinInfo(FieldSelector field) {
+            this.joinType = null;
+            this.field = field;
+            this.other = null;
+        }
+        public JoinInfo(FieldSelector field, String otherStream, JoinInfo otherStreamJoinInfo,  JoinType joinType) {
+            this.joinType = joinType;
+            this.field = field;
+            this.other = new FieldSelector(otherStream, otherStreamJoinInfo.field.getOutputName() );
+        }
+
+        public FieldSelector getJoinField() {
+            return field;
+        }
+
+        public String getOtherStream() {
+            return other.getStreamName();
+        }
+
+        public String[] getOtherField() {
+            return other.getField();
+        }
+
+        public JoinType getJoinType() {
+            return joinType;
+        }
+
+    } // class JoinInfo
+
+    // Join helper to concat fields to the record
+    protected class ResultRecord {
+
+        ArrayList<Tuple> tupleList = new ArrayList<>(); // contains one Tuple per Stream being joined
+        ArrayList<Object> outFields = null; // refs to fields that will be part of output fields
+
+        // 'generateOutputFields' enables us to avoid projection unless it is the final stream being joined
+        public ResultRecord(Tuple tuple, boolean generateOutputFields) {
+            tupleList.add(tuple);
+            if(generateOutputFields) {
+                outFields = doProjection(tupleList, outputFields);
+            }
+        }
+
+        public ResultRecord(ResultRecord lhs, Tuple rhs, boolean generateOutputFields) {
+            if(lhs!=null)
+                tupleList.addAll(lhs.tupleList);
+            if(rhs!=null)
+                tupleList.add(rhs);
+            if(generateOutputFields) {
+                outFields = doProjection(tupleList, outputFields);
+            }
+        }
+
+        public ArrayList<Object> getOutputFields() {
+            return outFields;
+        }
+
+
+        // 'stream' cannot be null,
+        public Object getField(FieldSelector fieldSelector) {
+            for (Tuple tuple : tupleList) {
+                Object result = lookupField(fieldSelector, tuple);
+                if (result!=null)
+                    return result;
+            }
+            return null;
+        }
+    }
+
+    protected class JoinAccumulator {
+        ArrayList<ResultRecord> records = new ArrayList<>();
+
+        public void insert(ResultRecord tuple) {
+            records.add( tuple );
+        }
+
+        public Collection<ResultRecord> getRecords() {
+            return records;
+        }
+    }
+
+    // Performs projection on the tuples based on 'projectionFields'
+    protected ArrayList<Object> doProjection(ArrayList<Tuple> tuples, FieldSelector[] projectionFields) {
+        ArrayList<Object> result = new ArrayList<>(projectionFields.length);
+        // Todo: optimize this computation... perhaps inner loop can be outside to avoid rescanning tuples
+        for ( int i = 0; i < projectionFields.length; i++ ) {
+            boolean missingField = true;
+            for ( Tuple tuple : tuples ) {
+                Object field = lookupField(projectionFields[i], tuple ) ;
+                if (field != null) {
+                    result.add(field);
+                    missingField=false;
+                    break;
+                }
+            }
+            if(missingField) { // add a null for missing fields (usually in case of outer joins)
+                result.add(null);
+            }
+        }
+        return result;
+    }
+
+    protected static class FieldSelector implements Serializable {
+        String streamName;    // can be null;
+        String[] field;       // nested field "x.y.z"  becomes => String["x","y","z"]
+        String outputName;    // either "stream1:x.y.z" or "x.y.z" depending on whether stream name is present.
+
+        public FieldSelector(String fieldDescriptor)  {  // sample fieldDescriptor = "stream1:x.y.z"
+            int pos = fieldDescriptor.indexOf(':');
+
+            if (pos>0) {  // stream name is specified
+                streamName = fieldDescriptor.substring(0,pos).trim();
+                outputName = fieldDescriptor.trim();
+                field =  fieldDescriptor.substring(pos+1, fieldDescriptor.length()).split("\\.");
+                return;
+            }
+
+            // stream name unspecified
+            streamName = null;
+            if(pos==0) {
+                outputName = fieldDescriptor.substring(1, fieldDescriptor.length() ).trim();
+
+            } else if (pos<0) {
+                outputName = fieldDescriptor.trim();
+            }
+            field =  outputName.split("\\.");
+        }
+
+        /**
+         * @param stream name of stream
+         * @param fieldDescriptor  Simple fieldDescriptor like "x.y.z" and w/o a 'stream1:' stream qualifier.
+         */
+        public FieldSelector(String stream, String fieldDescriptor)  {
+            this(fieldDescriptor);
+            if(fieldDescriptor.indexOf(":")>=0) {
+                throw new IllegalArgumentException("Not expecting stream qualifier ':' in '" + fieldDescriptor
+                        + "'. Stream name '" + stream +  "' is implicit in this context");
+            }
+            this.streamName = stream;
+        }
+
+        public FieldSelector(String stream, String[] field)  {
+            this( stream, String.join(".", field) );
+        }
+
+
+        public String getStreamName() {
+            return streamName;
+        }
+
+        public String[] getField() {
+            return field;
+        }
+
+        public String getOutputName() {
+            return toString();
+        }
+
+        @Override
+        public String toString() {
+            return outputName;
+        }
+    }
+
+    // Extract the field from tuple. Field may be nested field (x.y.z)
+    protected Object lookupField(FieldSelector fieldSelector, Tuple tuple) {
+
+        // very stream name matches, it stream name was specified
+        if ( fieldSelector.streamName!=null &&
+                !fieldSelector.streamName.equalsIgnoreCase( getStreamSelector(tuple) ) ) {
+            return null;
+        }
+
+        Object curr = null;
+        for (int i=0; i < fieldSelector.field.length; i++) {
+            if (i==0) {
+                if (tuple.contains(fieldSelector.field[i]) )
+                    curr = tuple.getValueByField(fieldSelector.field[i]);
+                else
+                    return null;
+            }  else  {
+                curr = ((Map) curr).get(fieldSelector.field[i]);
+                if (curr==null)
+                    return null;
+            }
+        }
+        return curr;
+    }
+
+    // Boilerplate overrides to cast result from base type to JoinBolt, so user doesn't have to
+    // down cast when invoking these methods
+
+    @Override
+    public JoinBolt withWindow(Count windowLength, Count slidingInterval) {
+        return (JoinBolt) super.withWindow(windowLength, slidingInterval);
+    }
+
+    @Override
+    public JoinBolt withWindow(Count windowLength, Duration slidingInterval) {
+        return (JoinBolt) super.withWindow(windowLength, slidingInterval);
+    }
+
+    @Override
+    public JoinBolt withWindow(Duration windowLength, Count slidingInterval) {
+        return (JoinBolt) super.withWindow(windowLength, slidingInterval);
+    }
+
+    @Override
+    public JoinBolt withWindow(Duration windowLength, Duration slidingInterval) {
+        return (JoinBolt) super.withWindow(windowLength, slidingInterval);
+    }
+
+    @Override
+    public JoinBolt withWindow(Count windowLength) {
+        return (JoinBolt) super.withWindow(windowLength);
+    }
+
+    @Override
+    public JoinBolt withWindow(Duration windowLength) {
+        return (JoinBolt) super.withWindow(windowLength);
+    }
+
+    @Override
+    public JoinBolt withTumblingWindow(Count count) {
+        return (JoinBolt) super.withTumblingWindow(count);
+    }
+
+    @Override
+    public JoinBolt withTumblingWindow(Duration duration) {
+        return (JoinBolt) super.withTumblingWindow(duration);
+    }
+
+    @Override
+    public JoinBolt withTimestampField(String fieldName) {
+        return (JoinBolt) super.withTimestampField(fieldName);
+    }
+
+    @Override
+    public JoinBolt withTimestampExtractor(TimestampExtractor timestampExtractor) {
+        return (JoinBolt) super.withTimestampExtractor(timestampExtractor);
+    }
+
+    @Override
+    public JoinBolt withLateTupleStream(String streamId) {
+        return (JoinBolt) super.withLateTupleStream(streamId);
+    }
+
+    @Override
+    public BaseWindowedBolt withLag(Duration duration) {
+        return (JoinBolt) super.withLag(duration);
+    }
+
+    @Override
+    public BaseWindowedBolt withWatermarkInterval(Duration interval) {
+        return (JoinBolt) super.withWatermarkInterval(interval);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/callback/DefaultWatcherCallBack.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/callback/DefaultWatcherCallBack.java b/storm-client/src/jvm/org/apache/storm/callback/DefaultWatcherCallBack.java
new file mode 100644
index 0000000..043dd0c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/callback/DefaultWatcherCallBack.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.callback;
+
+import org.apache.storm.zookeeper.ZkEventTypes;
+import org.apache.storm.zookeeper.ZkKeeperStates;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DefaultWatcherCallBack implements WatcherCallBack {
+
+    private static Logger LOG = LoggerFactory.getLogger(DefaultWatcherCallBack.class);
+
+    @Override
+    public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+        LOG.debug("Zookeeper state update:  {}, {}, {}", ZkKeeperStates.getStateName(state), ZkEventTypes.getTypeName(type), path);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/callback/WatcherCallBack.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/callback/WatcherCallBack.java b/storm-client/src/jvm/org/apache/storm/callback/WatcherCallBack.java
new file mode 100644
index 0000000..41a50ec
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/callback/WatcherCallBack.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.callback;
+
+import org.apache.zookeeper.Watcher;
+
+public interface WatcherCallBack {
+    public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java b/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
new file mode 100644
index 0000000..75b0e99
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.callback;
+
+import org.apache.zookeeper.Watcher;
+
+public interface ZKStateChangedCallback {
+    public void changed(Watcher.Event.EventType type, String path);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/clojure/ClojureBolt.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/clojure/ClojureBolt.java b/storm-client/src/jvm/org/apache/storm/clojure/ClojureBolt.java
new file mode 100644
index 0000000..60300e2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/clojure/ClojureBolt.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.clojure;
+
+import org.apache.storm.coordination.CoordinatedBolt.FinishedCallback;
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.task.IBolt;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import clojure.lang.IFn;
+import clojure.lang.PersistentArrayMap;
+import clojure.lang.Keyword;
+import clojure.lang.Symbol;
+import clojure.lang.RT;
+import org.apache.storm.utils.Utils;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+public class ClojureBolt implements IRichBolt, FinishedCallback {
+    Map<String, StreamInfo> _fields;
+    List<String> _fnSpec;
+    List<String> _confSpec;
+    List<Object> _params;
+    
+    IBolt _bolt;
+    
+    public ClojureBolt(List fnSpec, List confSpec, List<Object> params, Map<String, StreamInfo> fields) {
+        _fnSpec = fnSpec;
+        _confSpec = confSpec;
+        _params = params;
+        _fields = fields;
+    }
+
+    @Override
+    public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
+        IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1));
+        try {
+            IFn preparer = (IFn) hof.applyTo(RT.seq(_params));
+            final Map<Keyword,Object> collectorMap = new PersistentArrayMap( new Object[] {
+                Keyword.intern(Symbol.create("output-collector")), collector,
+                Keyword.intern(Symbol.create("context")), context});
+            List<Object> args = new ArrayList<Object>() {{
+                add(stormConf);
+                add(context);
+                add(collectorMap);
+            }};
+            
+            _bolt = (IBolt) preparer.applyTo(RT.seq(args));
+            //this is kind of unnecessary for clojure
+            try {
+                _bolt.prepare(stormConf, context, collector);
+            } catch(AbstractMethodError ame) {
+                
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        _bolt.execute(input);
+    }
+
+    @Override
+    public void cleanup() {
+            try {
+                _bolt.cleanup();
+            } catch(AbstractMethodError ame) {
+                
+            }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        for(String stream: _fields.keySet()) {
+            StreamInfo info = _fields.get(stream);
+            declarer.declareStream(stream, info.is_direct(), new Fields(info.get_output_fields()));
+        }
+    }
+
+    @Override
+    public void finishedId(Object id) {
+        if(_bolt instanceof FinishedCallback) {
+            ((FinishedCallback) _bolt).finishedId(id);
+        }
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1));
+        try {
+            return (Map) hof.applyTo(RT.seq(_params));
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/clojure/ClojureSpout.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/clojure/ClojureSpout.java b/storm-client/src/jvm/org/apache/storm/clojure/ClojureSpout.java
new file mode 100644
index 0000000..372b306
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/clojure/ClojureSpout.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.clojure;
+
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.spout.ISpout;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+import clojure.lang.IFn;
+import clojure.lang.PersistentArrayMap;
+import clojure.lang.Keyword;
+import clojure.lang.Symbol;
+import clojure.lang.RT;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class ClojureSpout implements IRichSpout {
+    Map<String, StreamInfo> _fields;
+    List<String> _fnSpec;
+    List<String> _confSpec;
+    List<Object> _params;
+    
+    ISpout _spout;
+    
+    public ClojureSpout(List fnSpec, List confSpec, List<Object> params, Map<String, StreamInfo> fields) {
+        _fnSpec = fnSpec;
+        _confSpec = confSpec;
+        _params = params;
+        _fields = fields;
+    }
+    
+
+    @Override
+    public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
+        IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1));
+        try {
+            IFn preparer = (IFn) hof.applyTo(RT.seq(_params));
+            final Map<Keyword,Object> collectorMap = new PersistentArrayMap( new Object[] {
+                Keyword.intern(Symbol.create("output-collector")), collector,
+                Keyword.intern(Symbol.create("context")), context});
+            List<Object> args = new ArrayList<Object>() {{
+                add(conf);
+                add(context);
+                add(collectorMap);
+            }};
+            
+            _spout = (ISpout) preparer.applyTo(RT.seq(args));
+            //this is kind of unnecessary for clojure
+            try {
+                _spout.open(conf, context, collector);
+            } catch(AbstractMethodError ame) {
+                
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            _spout.close();
+        } catch(AbstractMethodError ame) {
+                
+        }
+    }
+
+    @Override
+    public void nextTuple() {
+        try {
+            _spout.nextTuple();
+        } catch(AbstractMethodError ame) {
+                
+        }
+
+    }
+
+    @Override
+    public void ack(Object msgId) {
+        try {
+            _spout.ack(msgId);
+        } catch(AbstractMethodError ame) {
+                
+        }
+
+    }
+
+    @Override
+    public void fail(Object msgId) {
+        try {
+            _spout.fail(msgId);
+        } catch(AbstractMethodError ame) {
+                
+        }
+
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        for(String stream: _fields.keySet()) {
+            StreamInfo info = _fields.get(stream);
+            declarer.declareStream(stream, info.is_direct(), new Fields(info.get_output_fields()));
+        }
+    }
+    
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1));
+        try {
+            return (Map) hof.applyTo(RT.seq(_params));
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void activate() {
+        try {
+            _spout.activate();
+        } catch(AbstractMethodError ame) {
+                
+        }
+    }
+
+    @Override
+    public void deactivate() {
+        try {
+            _spout.deactivate();
+        } catch(AbstractMethodError ame) {
+                
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/clojure/RichShellBolt.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/clojure/RichShellBolt.java b/storm-client/src/jvm/org/apache/storm/clojure/RichShellBolt.java
new file mode 100644
index 0000000..6de5637
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/clojure/RichShellBolt.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.clojure;
+
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.task.ShellBolt;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import java.util.Map;
+
+public class RichShellBolt extends ShellBolt implements IRichBolt {
+    private Map<String, StreamInfo> _outputs;
+    
+    public RichShellBolt(String[] command, Map<String, StreamInfo> outputs) {
+        super(command);
+        _outputs = outputs;
+    }
+    
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        for(String stream: _outputs.keySet()) {
+            StreamInfo def = _outputs.get(stream);
+            if(def.is_direct()) {
+                declarer.declareStream(stream, true, new Fields(def.get_output_fields()));
+            } else {
+                declarer.declareStream(stream, new Fields(def.get_output_fields()));                
+            }
+        }
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }    
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/clojure/RichShellSpout.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/clojure/RichShellSpout.java b/storm-client/src/jvm/org/apache/storm/clojure/RichShellSpout.java
new file mode 100644
index 0000000..9fb7e73
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/clojure/RichShellSpout.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.clojure;
+
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.spout.ShellSpout;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import java.util.Map;
+
+public class RichShellSpout extends ShellSpout implements IRichSpout {
+    private Map<String, StreamInfo> _outputs;
+
+    public RichShellSpout(String[] command, Map<String, StreamInfo> outputs) {
+        super(command);
+        _outputs = outputs;
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        for(String stream: _outputs.keySet()) {
+            StreamInfo def = _outputs.get(stream);
+            if(def.is_direct()) {
+                declarer.declareStream(stream, true, new Fields(def.get_output_fields()));
+            } else {
+                declarer.declareStream(stream, new Fields(def.get_output_fields()));
+            }
+        }
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }
+}


[44/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateContext.java b/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
new file mode 100644
index 0000000..9ad6a92
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.cluster;
+
+/**
+ * This class is intended to provide runtime-context to StateStorageFactory
+ * implementors, giving information such as what daemon is creating it.
+ */
+public class ClusterStateContext {
+    
+    private DaemonType daemonType;
+
+    public ClusterStateContext() {
+        daemonType = DaemonType.UNKNOWN;
+    }
+    
+    public ClusterStateContext(DaemonType daemonType) {
+        this.daemonType = daemonType;
+    }
+    
+    public DaemonType getDaemonType() {
+        return daemonType;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateListener.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateListener.java b/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateListener.java
new file mode 100644
index 0000000..30e467e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ClusterStateListener.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+public interface ClusterStateListener {
+    void stateChanged(ConnectionState newState);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java
new file mode 100644
index 0000000..a800b07
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ClusterUtils {
+
+    public static final String ZK_SEPERATOR = "/";
+
+    public static final String ASSIGNMENTS_ROOT = "assignments";
+    public static final String CODE_ROOT = "code";
+    public static final String STORMS_ROOT = "storms";
+    public static final String SUPERVISORS_ROOT = "supervisors";
+    public static final String WORKERBEATS_ROOT = "workerbeats";
+    public static final String BACKPRESSURE_ROOT = "backpressure";
+    public static final String ERRORS_ROOT = "errors";
+    public static final String BLOBSTORE_ROOT = "blobstore";
+    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
+    public static final String NIMBUSES_ROOT = "nimbuses";
+    public static final String CREDENTIALS_ROOT = "credentials";
+    public static final String LOGCONFIG_ROOT = "logconfigs";
+    public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
+
+    public static final String ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
+    public static final String STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
+    public static final String SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
+    public static final String WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
+    public static final String BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
+    public static final String ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
+    public static final String BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
+    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
+    public static final String NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
+    public static final String CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
+    public static final String LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
+    public static final String PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
+
+    // A singleton instance allows us to mock delegated static methods in our
+    // tests by subclassing.
+    private static final ClusterUtils INSTANCE = new ClusterUtils();
+    private static ClusterUtils _instance = INSTANCE;
+
+    /**
+     * Provide an instance of this class for delegates to use. To mock out delegated methods, provide an instance of a subclass that overrides the
+     * implementation of the delegated method.
+     *
+     * @param u a Cluster instance
+     */
+    public static void setInstance(ClusterUtils u) {
+        _instance = u;
+    }
+
+    /**
+     * Resets the singleton instance to the default. This is helpful to reset the class to its original functionality when mocking is no longer desired.
+     */
+    public static void resetInstance() {
+        _instance = INSTANCE;
+    }
+
+    public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
+        List<ACL> aclList = null;
+        String payload = (String) topoConf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
+        if (Utils.isZkAuthenticationConfiguredTopology(topoConf)) {
+            aclList = new ArrayList<>();
+            ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
+            aclList.add(acl1);
+            ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
+            aclList.add(acl2);
+        }
+        return aclList;
+    }
+
+    public static String supervisorPath(String id) {
+        return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String assignmentPath(String id) {
+        return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String blobstorePath(String key) {
+        return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
+    }
+
+    public static String blobstoreMaxKeySequenceNumberPath(String key) {
+        return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
+    }
+
+    public static String nimbusPath(String id) {
+        return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String stormPath(String id) {
+        return STORMS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String workerbeatStormRoot(String stormId) {
+        return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String workerbeatPath(String stormId, String node, Long port) {
+        return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+    }
+
+    public static String backpressureStormRoot(String stormId) {
+        return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String backpressurePath(String stormId, String node, Long port) {
+        return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+    }
+
+    public static String errorStormRoot(String stormId) {
+        return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String errorPath(String stormId, String componentId) {
+        try {
+            return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
+        } catch (UnsupportedEncodingException e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    public static String lastErrorPath(String stormId, String componentId) {
+        return errorPath(stormId, componentId) + "-last-error";
+    }
+
+    public static String credentialsPath(String stormId) {
+        return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String logConfigPath(String stormId) {
+        return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String profilerConfigPath(String stormId) {
+        return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
+        return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
+    }
+
+    public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz) {
+        if (serialized != null) {
+            return Utils.deserialize(serialized, clazz);
+        }
+        return null;
+    }
+
+    /**
+     * Ensures that we only return heartbeats for executors assigned to this worker
+     * @param executors
+     * @param workerHeartbeat
+     * @return
+     */
+    public static Map<ExecutorInfo, ExecutorBeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
+        Map<ExecutorInfo, ExecutorBeat> executorWhb = new HashMap<>();
+        Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
+        for (ExecutorInfo executor : executors) {
+            if (executorStatsMap.containsKey(executor)) {
+                int time = workerHeartbeat.get_time_secs();
+                int uptime = workerHeartbeat.get_uptime_secs();
+                ExecutorStats executorStats = workerHeartbeat.get_executor_stats().get(executor);
+                ExecutorBeat executorBeat = new ExecutorBeat(time, uptime, executorStats);
+                executorWhb.put(executor, executorBeat);
+            }
+        }
+        return executorWhb;
+    }
+
+    public IStormClusterState mkStormClusterStateImpl(Object stateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+        if (stateStorage instanceof IStateStorage) {
+            return new StormClusterStateImpl((IStateStorage) stateStorage, acls, context, false);
+        } else {
+            IStateStorage Storage = _instance.mkStateStorageImpl((Map) stateStorage, (Map) stateStorage, acls, context);
+            return new StormClusterStateImpl(Storage, acls, context, true);
+        }
+    }
+
+    public IStateStorage mkStateStorageImpl(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+        String className = null;
+        IStateStorage stateStorage = null;
+        if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
+            className = (String) config.get(Config.STORM_CLUSTER_STATE_STORE);
+        } else {
+            className = "org.apache.storm.cluster.ZKStateStorageFactory";
+        }
+        Class clazz = Class.forName(className);
+        StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
+        stateStorage = storageFactory.mkStore(config, auth_conf, acls, context);
+        return stateStorage;
+    }
+
+    public static IStateStorage mkStateStorage(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+        return _instance.mkStateStorageImpl(config, auth_conf, acls, context);
+    }
+
+    public static IStormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+        return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
+    }
+
+    public static String stringifyError(Throwable error) {
+        StringWriter result = new StringWriter();
+        PrintWriter printWriter = new PrintWriter(result);
+        error.printStackTrace(printWriter);
+        return result.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/ConnectionState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ConnectionState.java b/storm-client/src/jvm/org/apache/storm/cluster/ConnectionState.java
new file mode 100644
index 0000000..ec5e97d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ConnectionState.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+public enum ConnectionState {
+    CONNECTED,
+    RECONNECTED,
+    LOST
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/DaemonType.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/DaemonType.java b/storm-client/src/jvm/org/apache/storm/cluster/DaemonType.java
new file mode 100644
index 0000000..4589be3
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/DaemonType.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.cluster;
+
+public enum DaemonType {
+    SUPERVISOR,
+    NIMBUS,
+    WORKER,
+    PACEMAKER,
+    UNKNOWN
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ExecutorBeat.java b/storm-client/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
new file mode 100644
index 0000000..b32615e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import org.apache.storm.generated.ExecutorStats;
+
+public class ExecutorBeat {
+    private final int timeSecs;
+    private final int uptime;
+    private final ExecutorStats stats;
+
+    public ExecutorBeat(int timeSecs, int uptime, ExecutorStats stats) {
+        this.timeSecs = timeSecs;
+        this.uptime = uptime;
+        this.stats = stats;
+    }
+
+    public int getTimeSecs() {
+        return timeSecs;
+    }
+
+    public int getUptime() {
+        return uptime;
+    }
+
+    public ExecutorStats getStats() {
+        return stats;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java b/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java
new file mode 100644
index 0000000..aa731ff
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java
@@ -0,0 +1,219 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * StateStorage provides the API for the pluggable state store used by the
+ * Storm daemons. Data is stored in path/value format, and the store supports
+ * listing sub-paths at a given path.
+ * All data should be available across all nodes with eventual consistency.
+ *
+ * IMPORTANT NOTE:
+ * Heartbeats have different api calls used to interact with them. The root
+ * path (/) may or may not be the same as the root path for the other api calls.
+ *
+ * For example, performing these two calls:
+ *     set_data("/path", data, acls);
+ *     void set_worker_hb("/path", heartbeat, acls);
+ * may or may not cause a collision in "/path".
+ * Never use the same paths with the *_hb* methods as you do with the others.
+ */
+public interface IStateStorage {
+    
+    /**
+     * Registers a callback function that gets called when CuratorEvents happen.
+     * @param callback is a clojure IFn that accepts the type - translated to
+     * clojure keyword as in zookeeper - and the path: (callback type path)
+     * @return is an id that can be passed to unregister(...) to unregister the
+     * callback.
+     */
+    String register(ZKStateChangedCallback callback);
+
+    /**
+     * Unregisters a callback function that was registered with register(...).
+     * @param id is the String id that was returned from register(...).
+     */
+    void unregister(String id);
+
+    /**
+     * Path will be appended with a monotonically increasing integer, a new node
+     * will be created there, and data will be put at that node.
+     * @param path The path that the monotonically increasing integer suffix will
+     * be added to.
+     * @param data The data that will be written at the suffixed path's node.
+     * @param acls The acls to apply to the path. May be null.
+     * @return The path with the integer suffix appended.
+     */
+    String create_sequential(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Creates nodes for path and all its parents. Path elements are separated by
+     * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
+     * @param path The path to create, along with all its parents.
+     * @param acls The acls to apply to the path. May be null.
+     * @return path
+     */
+    void mkdirs(String path, List<ACL> acls);
+
+    /**
+     * Deletes the node at a given path, and any child nodes that may exist.
+     * @param path The path to delete
+     */
+    void delete_node(String path);
+
+    /**
+     * Creates an ephemeral node at path. Ephemeral nodes are destroyed
+     * by the store when the client disconnects.
+     * @param path The path where a node will be created.
+     * @param data The data to be written at the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Gets the 'version' of the node at a path. Optionally sets a watch
+     * on that node. The version should increase whenever a write happens.
+     * @param path The path to get the version of.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The integer version of this node.
+     */
+    Integer get_version(String path, boolean watch) throws Exception;
+
+    /**
+     * Check if a node exists and optionally set a watch on the path.
+     * @param path The path to check for the existence of a node.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return Whether or not a node exists at path.
+     */
+    boolean node_exists(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_children(String path, boolean watch);
+
+    /**
+     * Close the connection to the data store.
+     */
+    void close();
+
+    /**
+     * Set the value of the node at path to data.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_data(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the data from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The data at the node.
+     */
+    byte[] get_data(String path, boolean watch);
+
+    /**
+     * Get the data at the node along with its version. Data is returned
+     * in an Map with the keys data and version.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return the data with a version
+     */
+    VersionedData<byte[]> get_data_with_version(String path, boolean watch);
+
+    /**
+     * Write a worker heartbeat at the path.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_worker_hb(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the heartbeat from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The heartbeat at the node.
+     */
+    byte[] get_worker_hb(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path. This is similar to get_children, but must be used for
+     * any nodes
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_worker_hb_children(String path, boolean watch);
+
+    /**
+     * Deletes the heartbeat at a given path, and any child nodes that may exist.
+     * @param path The path to delete.
+     */
+    void delete_worker_hb(String path);
+
+    /**
+     * Add a StateStorageListener to the connection.
+     * @param listener A StateStorageListener to handle changing cluster state
+     * events.
+     */
+    void add_listener(final ConnectionStateListener listener);
+
+    /**
+     * Force consistency on a path. Any writes committed on the path before
+     * this call will be completely propagated when it returns.
+     * @param path The path to synchronize.
+     */
+    void sync_path(String path);
+
+    /**
+     * Allows us to delete the znodes within /storm/blobstore/key_name
+     * whose znodes start with the corresponding nimbusHostPortInfo
+     * @param path /storm/blobstore/key_name
+     * @param nimbusHostPortInfo Contains the host port information of
+     * a nimbus node.
+     */
+    void delete_node_blobstore(String path, String nimbusHostPortInfo);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java
new file mode 100644
index 0000000..704c9e5
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.storm.generated.Assignment;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.Credentials;
+import org.apache.storm.generated.ErrorInfo;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.LogConfig;
+import org.apache.storm.generated.NimbusSummary;
+import org.apache.storm.generated.NodeInfo;
+import org.apache.storm.generated.ProfileRequest;
+import org.apache.storm.generated.StormBase;
+import org.apache.storm.generated.SupervisorInfo;
+import org.apache.storm.nimbus.NimbusInfo;
+
+public interface IStormClusterState {
+    public List<String> assignments(Runnable callback);
+
+    public Assignment assignmentInfo(String stormId, Runnable callback);
+
+    public VersionedData<Assignment> assignmentInfoWithVersion(String stormId, Runnable callback);
+
+    public Integer assignmentVersion(String stormId, Runnable callback) throws Exception;
+
+    public List<String> blobstoreInfo(String blobKey);
+
+    public List<NimbusSummary> nimbuses();
+
+    public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
+
+    public List<String> activeStorms();
+
+    /**
+     * Get a storm base for a topology
+     * @param stormId the id of the topology
+     * @param callback something to call if the data changes (best effort)
+     * @return the StormBase or null if it is not alive.
+     */
+    public StormBase stormBase(String stormId, Runnable callback);
+
+    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
+
+    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo);
+
+    public List<ProfileRequest> getTopologyProfileRequests(String stormId);
+
+    public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
+
+    public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
+
+    public Map<ExecutorInfo, ExecutorBeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+
+    public List<String> supervisors(Runnable callback);
+
+    public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
+
+    public void setupHeatbeats(String stormId);
+
+    public void teardownHeartbeats(String stormId);
+
+    public void teardownTopologyErrors(String stormId);
+
+    public List<String> heartbeatStorms();
+
+    public List<String> errorTopologies();
+
+    public List<String> backpressureTopologies();
+
+    public void setTopologyLogConfig(String stormId, LogConfig logConfig);
+
+    public LogConfig topologyLogConfig(String stormId, Runnable cb);
+
+    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
+
+    public void removeWorkerHeartbeat(String stormId, String node, Long port);
+
+    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
+
+    public void workerBackpressure(String stormId, String node, Long port, boolean on);
+
+    public boolean topologyBackpressure(String stormId, Runnable callback);
+
+    public void setupBackpressure(String stormId);
+
+    public void removeBackpressure(String stormId);
+
+    public void removeWorkerBackpressure(String stormId, String node, Long port);
+
+    public void activateStorm(String stormId, StormBase stormBase);
+
+    public void updateStorm(String stormId, StormBase newElems);
+
+    public void removeStormBase(String stormId);
+
+    public void setAssignment(String stormId, Assignment info);
+
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
+
+    public List<String> activeKeys();
+
+    public List<String> blobstore(Runnable callback);
+
+    public void removeStorm(String stormId);
+
+    public void removeBlobstoreKey(String blobKey);
+
+    public void removeKeyVersion(String blobKey);
+
+    public void reportError(String stormId, String componentId, String node, Long port, Throwable error);
+
+    public List<ErrorInfo> errors(String stormId, String componentId);
+
+    public ErrorInfo lastError(String stormId, String componentId);
+
+    public void setCredentials(String stormId, Credentials creds, Map<String, Object> topoConf) throws NoSuchAlgorithmException;
+
+    public Credentials credentials(String stormId, Runnable callback);
+
+    public void disconnect();
+    
+    /**
+     * @return All of the supervisors with the ID as the key
+     */
+    default Map<String, SupervisorInfo> allSupervisorInfo() {
+        return allSupervisorInfo(null);
+    }
+
+    /**
+     * @param callback be alerted if the list of supervisors change
+     * @return All of the supervisors with the ID as the key
+     */
+    default Map<String, SupervisorInfo> allSupervisorInfo(Runnable callback) {
+        Map<String, SupervisorInfo> ret = new HashMap<>();
+        for (String id: supervisors(callback)) {
+            ret.put(id, supervisorInfo(id));
+        }
+        return ret;
+    }
+    
+    /**
+     * Get a topology ID from the name of a topology
+     * @param topologyName the name of the topology to look for
+     * @return the id of the topology or null if it is not alive.
+     */
+    default Optional<String> getTopoId(final String topologyName) {
+        String ret = null;
+        for (String topoId: activeStorms()) {
+            String name = stormBase(topoId, null).get_name();
+            if (topologyName.equals(name)) {
+                ret = topoId;
+                break;
+            }
+        }
+        return Optional.ofNullable(ret);
+    }
+    
+    default Map<String, Assignment> topologyAssignments() {
+        Map<String, Assignment> ret = new HashMap<>();
+        for (String topoId: assignments(null)) {
+            ret.put(topoId, assignmentInfo(topoId, null));
+        }
+        return ret;
+    }
+    
+    default Map<String, StormBase> topologyBases() {
+        Map<String, StormBase> stormBases = new HashMap<>();
+        for (String topologyId : activeStorms()) {
+            StormBase base = stormBase(topologyId, null);
+            if (base != null) { //rece condition with delete
+                stormBases.put(topologyId, base);
+            }
+        }
+        return stormBases;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
new file mode 100644
index 0000000..9295347
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.generated.*;
+import org.apache.storm.pacemaker.PacemakerClientPool;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PaceMakerStateStorage implements IStateStorage {
+
+    private static Logger LOG = LoggerFactory.getLogger(PaceMakerStateStorage.class);
+
+    private PacemakerClientPool pacemakerClientPool;
+    private IStateStorage stateStorage;
+    private static final int maxRetries = 10;
+
+    public PaceMakerStateStorage(PacemakerClientPool pacemakerClientPool, IStateStorage stateStorage) throws Exception {
+        this.pacemakerClientPool = pacemakerClientPool;
+        this.stateStorage = stateStorage;
+    }
+
+    @Override
+    public String register(ZKStateChangedCallback callback) {
+        return stateStorage.register(callback);
+    }
+
+    @Override
+    public void unregister(String id) {
+        stateStorage.unregister(id);
+    }
+
+    @Override
+    public String create_sequential(String path, byte[] data, List<ACL> acls) {
+        return stateStorage.create_sequential(path, data, acls);
+    }
+
+    @Override
+    public void mkdirs(String path, List<ACL> acls) {
+        stateStorage.mkdirs(path, acls);
+    }
+
+    @Override
+    public void delete_node(String path) {
+        stateStorage.delete_node(path);
+    }
+
+    @Override
+    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+        stateStorage.set_ephemeral_node(path, data, acls);
+    }
+
+    @Override
+    public Integer get_version(String path, boolean watch) throws Exception {
+        return stateStorage.get_version(path, watch);
+    }
+
+    @Override
+    public boolean node_exists(String path, boolean watch) {
+        return stateStorage.node_exists(path, watch);
+    }
+
+    @Override
+    public List<String> get_children(String path, boolean watch) {
+        return stateStorage.get_children(path, watch);
+    }
+
+    @Override
+    public void close() {
+        stateStorage.close();
+        pacemakerClientPool.close();
+    }
+
+    @Override
+    public void set_data(String path, byte[] data, List<ACL> acls) {
+        stateStorage.set_data(path, data, acls);
+    }
+
+    @Override
+    public byte[] get_data(String path, boolean watch) {
+        return stateStorage.get_data(path, watch);
+    }
+
+    @Override
+    public VersionedData<byte[]> get_data_with_version(String path, boolean watch) {
+        return stateStorage.get_data_with_version(path, watch);
+    }
+
+    @Override
+    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+        int retry = maxRetries;
+        while (true) {
+            try {
+                HBPulse hbPulse = new HBPulse();
+                hbPulse.set_id(path);
+                hbPulse.set_details(data);
+                HBMessage message = new HBMessage(HBServerMessageType.SEND_PULSE, HBMessageData.pulse(hbPulse));
+                HBMessage response = pacemakerClientPool.send(message);
+                if (response.get_type() != HBServerMessageType.SEND_PULSE_RESPONSE) {
+                    throw new HBExecutionException("Invalid Response Type");
+                }
+                LOG.debug("Successful set_worker_hb");
+                break;
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    throw Utils.wrapInRuntime(e);
+                }
+                retry--;
+                LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
+            }
+        }
+    }
+
+    @Override
+    public byte[] get_worker_hb(String path, boolean watch) {
+        int retry = maxRetries;
+        while (true) {
+            try {
+                byte[] ret = null;
+                int latest_time_secs = 0;
+                boolean got_response = false;
+
+                HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+                List<HBMessage> responses = pacemakerClientPool.sendAll(message);
+                for(HBMessage response : responses) {
+                    if (response.get_type() != HBServerMessageType.GET_PULSE_RESPONSE) {
+                        LOG.error("get_worker_hb: Invalid Response Type");
+                        continue;
+                    }
+                    // We got at least one GET_PULSE_RESPONSE message.
+                    got_response = true;
+                    byte[] details = response.get_data().get_pulse().get_details();
+                    if(details == null) {
+                        continue;
+                    }
+                    ClusterWorkerHeartbeat cwh = Utils.deserialize(details, ClusterWorkerHeartbeat.class);
+                    if(cwh != null && cwh.get_time_secs() > latest_time_secs) {
+                        latest_time_secs = cwh.get_time_secs();
+                        ret = details;
+                    }
+                }
+                if(!got_response) {
+                    throw new HBExecutionException("Failed to get a response.");
+                }
+                return ret;
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    throw Utils.wrapInRuntime(e);
+                }
+                retry--;
+                LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
+            }
+        }
+    }
+
+    @Override
+    public List<String> get_worker_hb_children(String path, boolean watch) {
+        int retry = maxRetries;
+        while (true) {
+            try {
+                HashSet<String> retSet = new HashSet<>();
+
+                HBMessage message = new HBMessage(HBServerMessageType.GET_ALL_NODES_FOR_PATH, HBMessageData.path(path));
+                List<HBMessage> responses = pacemakerClientPool.sendAll(message);
+                for(HBMessage response : responses) {
+                    if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
+                        LOG.error("get_worker_hb_children: Invalid Response Type");
+                        continue;
+                    }
+                    if(response.get_data().get_nodes().get_pulseIds() != null) {
+                        retSet.addAll(response.get_data().get_nodes().get_pulseIds());
+                    }
+                }
+
+                LOG.debug("Successful get_worker_hb");
+                return new ArrayList<>(retSet);
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    throw Utils.wrapInRuntime(e);
+                }
+                retry--;
+                LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry);
+            }
+        }
+    }
+
+    @Override
+    public void delete_worker_hb(String path) {
+        int retry = maxRetries;
+        boolean someSucceeded;
+        while (true) {
+            someSucceeded = false;
+            try {
+                HBMessage message = new HBMessage(HBServerMessageType.DELETE_PATH, HBMessageData.path(path));
+                List<HBMessage> responses = pacemakerClientPool.sendAll(message);
+                boolean allSucceeded = true;
+                for(HBMessage response : responses) {
+                    if (response.get_type() != HBServerMessageType.DELETE_PATH_RESPONSE) {
+                        LOG.debug("Failed to delete heartbeat {}", response);
+                        allSucceeded = false;
+                    }
+                    else {
+                        someSucceeded = true;
+                    }
+                }
+                if(allSucceeded) {
+                    break;
+                }
+                else {
+                    throw new HBExecutionException("Failed to delete from all pacemakers.");
+                }
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    if(someSucceeded) {
+                        LOG.warn("Unable to delete_worker_hb from every pacemaker.");
+                        break;
+                    }
+                    else {
+                        LOG.error("Unable to delete_worker_hb from any pacemaker.");
+                        throw Utils.wrapInRuntime(e);
+                    }
+                }
+                retry--;
+                LOG.debug("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
+            }
+        }
+    }
+
+    @Override
+    public void add_listener(ConnectionStateListener listener) {
+        stateStorage.add_listener(listener);
+    }
+
+    @Override
+    public void sync_path(String path) {
+        stateStorage.sync_path(path);
+    }
+
+    @Override
+    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+        stateStorage.delete_node_blobstore(path, nimbusHostPortInfo);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
new file mode 100644
index 0000000..3574506
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.storm.pacemaker.PacemakerClientPool;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+
+public class PaceMakerStateStorageFactory implements StateStorageFactory {
+    @Override
+    public IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) {
+        try {
+            ZKStateStorageFactory zkfact = new ZKStateStorageFactory();
+            IStateStorage zkState = zkfact.mkStore(config, auth_conf, acls, context);
+            return new PaceMakerStateStorage(new PacemakerClientPool(config), zkState);
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
new file mode 100644
index 0000000..0929750
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.zookeeper.data.ACL;
+
+public interface StateStorageFactory {
+
+    IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
new file mode 100644
index 0000000..51caad9
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -0,0 +1,735 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.*;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.generated.*;
+import org.apache.storm.nimbus.NimbusInfo;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.Time;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class StormClusterStateImpl implements IStormClusterState {
+
+    private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
+
+    private IStateStorage stateStorage;
+
+    private ConcurrentHashMap<String, Runnable> assignmentInfoCallback;
+    private ConcurrentHashMap<String, Runnable> assignmentInfoWithVersionCallback;
+    private ConcurrentHashMap<String, Runnable> assignmentVersionCallback;
+    private AtomicReference<Runnable> supervisorsCallback;
+    // we want to reigister a topo directory getChildren callback for all workers of this dir
+    private ConcurrentHashMap<String, Runnable> backPressureCallback;
+    private AtomicReference<Runnable> assignmentsCallback;
+    private ConcurrentHashMap<String, Runnable> stormBaseCallback;
+    private AtomicReference<Runnable> blobstoreCallback;
+    private ConcurrentHashMap<String, Runnable> credentialsCallback;
+    private ConcurrentHashMap<String, Runnable> logConfigCallback;
+
+    private List<ACL> acls;
+    private String stateId;
+    private boolean solo;
+
+    public StormClusterStateImpl(IStateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
+
+        this.stateStorage = StateStorage;
+        this.solo = solo;
+        this.acls = acls;
+
+        assignmentInfoCallback = new ConcurrentHashMap<>();
+        assignmentInfoWithVersionCallback = new ConcurrentHashMap<>();
+        assignmentVersionCallback = new ConcurrentHashMap<>();
+        supervisorsCallback = new AtomicReference<>();
+        backPressureCallback = new ConcurrentHashMap<>();
+        assignmentsCallback = new AtomicReference<>();
+        stormBaseCallback = new ConcurrentHashMap<>();
+        credentialsCallback = new ConcurrentHashMap<>();
+        logConfigCallback = new ConcurrentHashMap<>();
+        blobstoreCallback = new AtomicReference<>();
+
+        stateId = this.stateStorage.register(new ZKStateChangedCallback() {
+
+            public void changed(Watcher.Event.EventType type, String path) {
+                List<String> toks = tokenizePath(path);
+                int size = toks.size();
+                if (size >= 1) {
+                    String root = toks.get(0);
+                    if (root.equals(ClusterUtils.ASSIGNMENTS_ROOT)) {
+                        if (size == 1) {
+                            // set null and get the old value
+                            issueCallback(assignmentsCallback);
+                        } else {
+                            issueMapCallback(assignmentInfoCallback, toks.get(1));
+                            issueMapCallback(assignmentVersionCallback, toks.get(1));
+                            issueMapCallback(assignmentInfoWithVersionCallback, toks.get(1));
+                        }
+
+                    } else if (root.equals(ClusterUtils.SUPERVISORS_ROOT)) {
+                        issueCallback(supervisorsCallback);
+                    } else if (root.equals(ClusterUtils.BLOBSTORE_ROOT)) {
+                        issueCallback(blobstoreCallback);
+                    } else if (root.equals(ClusterUtils.STORMS_ROOT) && size > 1) {
+                        issueMapCallback(stormBaseCallback, toks.get(1));
+                    } else if (root.equals(ClusterUtils.CREDENTIALS_ROOT) && size > 1) {
+                        issueMapCallback(credentialsCallback, toks.get(1));
+                    } else if (root.equals(ClusterUtils.LOGCONFIG_ROOT) && size > 1) {
+                        issueMapCallback(logConfigCallback, toks.get(1));
+                    } else if (root.equals(ClusterUtils.BACKPRESSURE_ROOT) && size > 1) {
+                        issueMapCallback(backPressureCallback, toks.get(1));
+                    } else {
+                        LOG.error("{} Unknown callback for subtree {}", new RuntimeException("Unknown callback for this path"), path);
+                        Runtime.getRuntime().exit(30);
+                    }
+
+                }
+
+                return;
+            }
+
+        });
+
+        String[] pathlist = { ClusterUtils.ASSIGNMENTS_SUBTREE, 
+                              ClusterUtils.STORMS_SUBTREE, 
+                              ClusterUtils.SUPERVISORS_SUBTREE, 
+                              ClusterUtils.WORKERBEATS_SUBTREE,
+                              ClusterUtils.ERRORS_SUBTREE, 
+                              ClusterUtils.BLOBSTORE_SUBTREE, 
+                              ClusterUtils.NIMBUSES_SUBTREE, 
+                              ClusterUtils.LOGCONFIG_SUBTREE,
+                              ClusterUtils.BACKPRESSURE_SUBTREE };
+        for (String path : pathlist) {
+            this.stateStorage.mkdirs(path, acls);
+        }
+
+    }
+
+    protected void issueCallback(AtomicReference<Runnable> cb) {
+        Runnable callback = cb.getAndSet(null);
+        if (callback != null)
+            callback.run();
+    }
+
+    protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConcurrentHashMap, String key) {
+        Runnable callback = callbackConcurrentHashMap.remove(key);
+        if (callback != null)
+            callback.run();
+    }
+
+    @Override
+    public List<String> assignments(Runnable callback) {
+        if (callback != null) {
+            assignmentsCallback.set(callback);
+        }
+        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
+    }
+
+    @Override
+    public Assignment assignmentInfo(String stormId, Runnable callback) {
+        if (callback != null) {
+            assignmentInfoCallback.put(stormId, callback);
+        }
+        byte[] serialized = stateStorage.get_data(ClusterUtils.assignmentPath(stormId), callback != null);
+        return ClusterUtils.maybeDeserialize(serialized, Assignment.class);
+    }
+
+    @Override
+    public VersionedData<Assignment> assignmentInfoWithVersion(String stormId, Runnable callback) {
+        if (callback != null) {
+            assignmentInfoWithVersionCallback.put(stormId, callback);
+        }
+        Assignment assignment = null;
+        Integer version = 0;
+        VersionedData<byte[]> dataWithVersion = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
+        if (dataWithVersion != null) {
+            assignment = ClusterUtils.maybeDeserialize(dataWithVersion.getData(), Assignment.class);
+            version = dataWithVersion.getVersion();
+        }
+        return new VersionedData<Assignment>(version, assignment);
+    }
+
+    @Override
+    public Integer assignmentVersion(String stormId, Runnable callback) throws Exception {
+        if (callback != null) {
+            assignmentVersionCallback.put(stormId, callback);
+        }
+        return stateStorage.get_version(ClusterUtils.assignmentPath(stormId), callback != null);
+    }
+
+    // blobstore state
+    @Override
+    public List<String> blobstoreInfo(String blobKey) {
+        String path = ClusterUtils.blobstorePath(blobKey);
+        stateStorage.sync_path(path);
+        return stateStorage.get_children(path, false);
+    }
+
+    @Override
+    public List<NimbusSummary> nimbuses() {
+        List<NimbusSummary> nimbusSummaries = new ArrayList<>();
+        List<String> nimbusIds = stateStorage.get_children(ClusterUtils.NIMBUSES_SUBTREE, false);
+        for (String nimbusId : nimbusIds) {
+            byte[] serialized = stateStorage.get_data(ClusterUtils.nimbusPath(nimbusId), false);
+            // check for null which can exist because of a race condition in which nimbus nodes in zk may have been
+            // removed when connections are reconnected after getting children in the above line
+            if (serialized != null) {
+                NimbusSummary nimbusSummary = ClusterUtils.maybeDeserialize(serialized, NimbusSummary.class);
+                nimbusSummaries.add(nimbusSummary);
+            }
+        }
+        return nimbusSummaries;
+    }
+
+    @Override
+    public void addNimbusHost(final String nimbusId, final NimbusSummary nimbusSummary) {
+        // explicit delete for ephmeral node to ensure this session creates the entry.
+        stateStorage.delete_node(ClusterUtils.nimbusPath(nimbusId));
+        stateStorage.add_listener(new ConnectionStateListener() {
+            @Override
+            public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
+                LOG.info("Connection state listener invoked, zookeeper connection state has changed to {}", connectionState);
+                if (connectionState.equals(ConnectionState.RECONNECTED)) {
+                    LOG.info("Connection state has changed to reconnected so setting nimbuses entry one more time");
+                    // explicit delete for ephmeral node to ensure this session creates the entry.
+                    stateStorage.delete_node(ClusterUtils.nimbusPath(nimbusId));
+                    stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+                }
+
+            }
+        });
+
+        stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+    }
+
+    @Override
+    public List<String> activeStorms() {
+        return stateStorage.get_children(ClusterUtils.STORMS_SUBTREE, false);
+    }
+
+    @Override
+    public StormBase stormBase(String stormId, Runnable callback) {
+        if (callback != null) {
+            stormBaseCallback.put(stormId, callback);
+        }
+        return ClusterUtils.maybeDeserialize(stateStorage.get_data(ClusterUtils.stormPath(stormId), callback != null), StormBase.class);
+    }
+
+    @Override
+    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port) {
+        byte[] bytes = stateStorage.get_worker_hb(ClusterUtils.workerbeatPath(stormId, node, port), false);
+        return ClusterUtils.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class);
+
+    }
+
+    @Override
+    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo) {
+        List<ProfileRequest> requests = new ArrayList<>();
+        List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId);
+        for (ProfileRequest profileRequest : profileRequests) {
+            NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
+            if (nodeInfo1.equals(nodeInfo))
+                requests.add(profileRequest);
+        }
+        return requests;
+    }
+
+    @Override
+    public List<ProfileRequest> getTopologyProfileRequests(String stormId) {
+        List<ProfileRequest> profileRequests = new ArrayList<>();
+        String path = ClusterUtils.profilerConfigPath(stormId);
+        if (stateStorage.node_exists(path, false)) {
+            List<String> strs = stateStorage.get_children(path, false);
+            for (String str : strs) {
+                String childPath = path + ClusterUtils.ZK_SEPERATOR + str;
+                byte[] raw = stateStorage.get_data(childPath, false);
+                ProfileRequest request = ClusterUtils.maybeDeserialize(raw, ProfileRequest.class);
+                if (request != null)
+                    profileRequests.add(request);
+            }
+        }
+        return profileRequests;
+    }
+
+    @Override
+    public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest) {
+        ProfileAction profileAction = profileRequest.get_action();
+        String host = profileRequest.get_nodeInfo().get_node();
+        Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+        String path = ClusterUtils.profilerConfigPath(stormId, host, port, profileAction);
+        stateStorage.set_data(path, Utils.serialize(profileRequest), acls);
+    }
+
+    @Override
+    public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest) {
+        ProfileAction profileAction = profileRequest.get_action();
+        String host = profileRequest.get_nodeInfo().get_node();
+        Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+        String path = ClusterUtils.profilerConfigPath(stormId, host, port, profileAction);
+        stateStorage.delete_node(path);
+    }
+
+    /**
+     * need to take executor->node+port in explicitly so that we don't run into a situation where a long dead worker with a skewed clock overrides all the
+     * timestamps. By only checking heartbeats with an assigned node+port, and only reading executors from that heartbeat that are actually assigned, we avoid
+     * situations like that
+     * 
+     * @param stormId
+     * @param executorNodePort
+     * @return
+     */
+    @Override
+    public Map<ExecutorInfo, ExecutorBeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
+        Map<ExecutorInfo, ExecutorBeat> executorWhbs = new HashMap<>();
+
+        Map<NodeInfo, List<List<Long>>> nodePortExecutors = Utils.reverseMap(executorNodePort);
+
+        for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
+
+            String node = entry.getKey().get_node();
+            Long port = entry.getKey().get_port_iterator().next();
+            ClusterWorkerHeartbeat whb = getWorkerHeartbeat(stormId, node, port);
+            List<ExecutorInfo> executorInfoList = new ArrayList<>();
+            for (List<Long> list : entry.getValue()) {
+                executorInfoList.add(new ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue()));
+            }
+            if (whb != null)
+                executorWhbs.putAll(ClusterUtils.convertExecutorBeats(executorInfoList, whb));
+        }
+        return executorWhbs;
+    }
+
+    @Override
+    public List<String> supervisors(Runnable callback) {
+        if (callback != null) {
+            supervisorsCallback.set(callback);
+        }
+        return stateStorage.get_children(ClusterUtils.SUPERVISORS_SUBTREE, callback != null);
+    }
+
+    @Override
+    public SupervisorInfo supervisorInfo(String supervisorId) {
+        String path = ClusterUtils.supervisorPath(supervisorId);
+        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), SupervisorInfo.class);
+    }
+
+    @Override
+    public void setupHeatbeats(String stormId) {
+        stateStorage.mkdirs(ClusterUtils.workerbeatStormRoot(stormId), acls);
+    }
+
+    @Override
+    public void teardownHeartbeats(String stormId) {
+        try {
+            stateStorage.delete_worker_hb(ClusterUtils.workerbeatStormRoot(stormId));
+        } catch (Exception e) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, e)) {
+                // do nothing
+                LOG.warn("Could not teardown heartbeats for {}.", stormId);
+            } else {
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public void teardownTopologyErrors(String stormId) {
+        try {
+            stateStorage.delete_node(ClusterUtils.errorStormRoot(stormId));
+        } catch (Exception e) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, e)) {
+                // do nothing
+                LOG.warn("Could not teardown errors for {}.", stormId);
+            } else {
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public List<String> heartbeatStorms() {
+        return stateStorage.get_worker_hb_children(ClusterUtils.WORKERBEATS_SUBTREE, false);
+    }
+
+    @Override
+    public List<String> errorTopologies() {
+        return stateStorage.get_children(ClusterUtils.ERRORS_SUBTREE, false);
+    }
+
+    @Override
+    public List<String> backpressureTopologies() {
+        return stateStorage.get_children(ClusterUtils.BACKPRESSURE_SUBTREE, false);
+    }
+
+    @Override
+    public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
+        stateStorage.set_data(ClusterUtils.logConfigPath(stormId), Utils.serialize(logConfig), acls);
+    }
+
+    @Override
+    public LogConfig topologyLogConfig(String stormId, Runnable cb) {
+        if (cb != null){
+            logConfigCallback.put(stormId, cb);
+        }
+        String path = ClusterUtils.logConfigPath(stormId);
+        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
+    }
+
+    @Override
+    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
+        if (info != null) {
+            String path = ClusterUtils.workerbeatPath(stormId, node, port);
+            stateStorage.set_worker_hb(path, Utils.serialize(info), acls);
+        }
+    }
+
+    @Override
+    public void removeWorkerHeartbeat(String stormId, String node, Long port) {
+        String path = ClusterUtils.workerbeatPath(stormId, node, port);
+        stateStorage.delete_worker_hb(path);
+    }
+
+    @Override
+    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
+        String path = ClusterUtils.supervisorPath(supervisorId);
+        stateStorage.set_ephemeral_node(path, Utils.serialize(info), acls);
+    }
+
+    /**
+     * if znode exists and to be not on?, delete; if exists and on?, do nothing; if not exists and to be on?, create; if not exists and not on?, do nothing;
+     * 
+     * @param stormId
+     * @param node
+     * @param port
+     * @param on
+     */
+    @Override
+    public void workerBackpressure(String stormId, String node, Long port, boolean on) {
+        String path = ClusterUtils.backpressurePath(stormId, node, port);
+        boolean existed = stateStorage.node_exists(path, false);
+        if (existed) {
+            if (on == false)
+                stateStorage.delete_node(path);
+
+        } else {
+            if (on == true) {
+                stateStorage.set_ephemeral_node(path, null, acls);
+            }
+        }
+    }
+
+    /**
+     * Check whether a topology is in throttle-on status or not:
+     * if the backpresure/storm-id dir is not empty, this topology has throttle-on, otherwise throttle-off.
+     * 
+     * @param stormId
+     * @param callback
+     * @return
+     */
+    @Override
+    public boolean topologyBackpressure(String stormId, Runnable callback) {
+        if (callback != null) {
+            backPressureCallback.put(stormId, callback);
+        }
+        String path = ClusterUtils.backpressureStormRoot(stormId);
+        List<String> childrens = null;
+        if(stateStorage.node_exists(path, false)) {
+            childrens = stateStorage.get_children(path, callback != null);
+        } else {
+            childrens = new ArrayList<>();
+        }
+        return childrens.size() > 0;
+
+    }
+
+    @Override
+    public void setupBackpressure(String stormId) {
+        stateStorage.mkdirs(ClusterUtils.backpressureStormRoot(stormId), acls);
+    }
+
+    @Override
+    public void removeBackpressure(String stormId) {
+        try {
+            stateStorage.delete_node(ClusterUtils.backpressureStormRoot(stormId));
+        } catch (Exception e) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, e)) {
+                // do nothing
+                LOG.warn("Could not teardown backpressure node for {}.", stormId);
+            } else {
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public void removeWorkerBackpressure(String stormId, String node, Long port) {
+        String path = ClusterUtils.backpressurePath(stormId, node, port);
+        boolean existed = stateStorage.node_exists(path, false);
+        if (existed) {
+            stateStorage.delete_node(path);
+        }
+    }
+
+    @Override
+    public void activateStorm(String stormId, StormBase stormBase) {
+        String path = ClusterUtils.stormPath(stormId);
+        stateStorage.set_data(path, Utils.serialize(stormBase), acls);
+    }
+
+    /**
+     * To update this function due to APersistentMap/APersistentSet is clojure's structure
+     * 
+     * @param stormId
+     * @param newElems
+     */
+    @Override
+    public void updateStorm(String stormId, StormBase newElems) {
+
+        StormBase stormBase = stormBase(stormId, null);
+        if (stormBase.get_component_executors() != null) {
+
+            Map<String, Integer> newComponentExecutors = new HashMap<>();
+            Map<String, Integer> componentExecutors = newElems.get_component_executors();
+            // componentExecutors maybe be APersistentMap, which don't support "put"
+            for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
+                newComponentExecutors.put(entry.getKey(), entry.getValue());
+            }
+            for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
+                if (!componentExecutors.containsKey(entry.getKey())) {
+                    newComponentExecutors.put(entry.getKey(), entry.getValue());
+                }
+            }
+            if (newComponentExecutors.size() > 0)
+                newElems.set_component_executors(newComponentExecutors);
+        }
+
+        Map<String, DebugOptions> ComponentDebug = new HashMap<>();
+        Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
+
+        Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
+        /// oldComponentDebug.keySet()/ newComponentDebug.keySet() maybe be APersistentSet, which don't support addAll
+        Set<String> debugOptionsKeys = new HashSet<>();
+        debugOptionsKeys.addAll(oldComponentDebug.keySet());
+        debugOptionsKeys.addAll(newComponentDebug.keySet());
+        for (String key : debugOptionsKeys) {
+            boolean enable = false;
+            double samplingpct = 0;
+            if (oldComponentDebug.containsKey(key)) {
+                enable = oldComponentDebug.get(key).is_enable();
+                samplingpct = oldComponentDebug.get(key).get_samplingpct();
+            }
+            if (newComponentDebug.containsKey(key)) {
+                enable = newComponentDebug.get(key).is_enable();
+                samplingpct += newComponentDebug.get(key).get_samplingpct();
+            }
+            DebugOptions debugOptions = new DebugOptions();
+            debugOptions.set_enable(enable);
+            debugOptions.set_samplingpct(samplingpct);
+            ComponentDebug.put(key, debugOptions);
+        }
+        if (ComponentDebug.size() > 0) {
+            newElems.set_component_debug(ComponentDebug);
+        }
+
+        if (StringUtils.isBlank(newElems.get_name())) {
+            newElems.set_name(stormBase.get_name());
+        }
+        if (newElems.get_status() == null) {
+            newElems.set_status(stormBase.get_status());
+        }
+        if (newElems.get_num_workers() == 0) {
+            newElems.set_num_workers(stormBase.get_num_workers());
+        }
+        if (newElems.get_launch_time_secs() == 0) {
+            newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
+        }
+        if (StringUtils.isBlank(newElems.get_owner())) {
+            newElems.set_owner(stormBase.get_owner());
+        }
+        if (newElems.get_topology_action_options() == null) {
+            newElems.set_topology_action_options(stormBase.get_topology_action_options());
+        }
+        if (newElems.get_status() == null) {
+            newElems.set_status(stormBase.get_status());
+        }
+        stateStorage.set_data(ClusterUtils.stormPath(stormId), Utils.serialize(newElems), acls);
+    }
+
+    @Override
+    public void removeStormBase(String stormId) {
+        stateStorage.delete_node(ClusterUtils.stormPath(stormId));
+    }
+
+    @Override
+    public void setAssignment(String stormId, Assignment info) {
+        stateStorage.set_data(ClusterUtils.assignmentPath(stormId), Utils.serialize(info), acls);
+    }
+
+    @Override
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
+        String path = ClusterUtils.blobstorePath(key) + ClusterUtils.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
+        LOG.info("set-path: {}", path);
+        stateStorage.mkdirs(ClusterUtils.blobstorePath(key), acls);
+        stateStorage.delete_node_blobstore(ClusterUtils.blobstorePath(key), nimbusInfo.toHostPortString());
+        stateStorage.set_ephemeral_node(path, null, acls);
+    }
+
+    @Override
+    public List<String> activeKeys() {
+        return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, false);
+    }
+
+    // blobstore state
+    @Override
+    public List<String> blobstore(Runnable callback) {
+        if (callback != null) {
+            blobstoreCallback.set(callback);
+        }
+        stateStorage.sync_path(ClusterUtils.BLOBSTORE_SUBTREE);
+        return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, callback != null);
+
+    }
+
+    @Override
+    public void removeStorm(String stormId) {
+        stateStorage.delete_node(ClusterUtils.assignmentPath(stormId));
+        stateStorage.delete_node(ClusterUtils.credentialsPath(stormId));
+        stateStorage.delete_node(ClusterUtils.logConfigPath(stormId));
+        stateStorage.delete_node(ClusterUtils.profilerConfigPath(stormId));
+        removeStormBase(stormId);
+    }
+
+    @Override
+    public void removeBlobstoreKey(String blobKey) {
+        LOG.debug("remove key {}", blobKey);
+        stateStorage.delete_node(ClusterUtils.blobstorePath(blobKey));
+    }
+
+    @Override
+    public void removeKeyVersion(String blobKey) {
+        stateStorage.delete_node(ClusterUtils.blobstoreMaxKeySequenceNumberPath(blobKey));
+    }
+
+    @Override
+    public void reportError(String stormId, String componentId, String node, Long port, Throwable error) {
+
+        String path = ClusterUtils.errorPath(stormId, componentId);
+        String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
+        ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.stringifyError(error), Time.currentTimeSecs());
+        errorInfo.set_host(node);
+        errorInfo.set_port(port.intValue());
+        byte[] serData = Utils.serialize(errorInfo);
+        stateStorage.mkdirs(path, acls);
+        stateStorage.create_sequential(path + ClusterUtils.ZK_SEPERATOR + "e", serData, acls);
+        stateStorage.set_data(lastErrorPath, serData, acls);
+        List<String> childrens = stateStorage.get_children(path, false);
+
+        Collections.sort(childrens, new Comparator<String>() {
+            public int compare(String arg0, String arg1) {
+                return Long.compare(Long.parseLong(arg0.substring(1)), Long.parseLong(arg1.substring(1)));
+            }
+        });
+
+        while (childrens.size() > 10) {
+            stateStorage.delete_node(path + ClusterUtils.ZK_SEPERATOR + childrens.remove(0));
+        }
+    }
+
+    @Override
+    public List<ErrorInfo> errors(String stormId, String componentId) {
+        List<ErrorInfo> errorInfos = new ArrayList<>();
+        String path = ClusterUtils.errorPath(stormId, componentId);
+        if (stateStorage.node_exists(path, false)) {
+            List<String> childrens = stateStorage.get_children(path, false);
+            for (String child : childrens) {
+                String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
+                ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
+                if (errorInfo != null)
+                    errorInfos.add(errorInfo);
+            }
+        }
+        Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+            public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+                return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
+            }
+        });
+
+        return errorInfos;
+    }
+
+    @Override
+    public ErrorInfo lastError(String stormId, String componentId) {
+
+        String path = ClusterUtils.lastErrorPath(stormId, componentId);
+        if (stateStorage.node_exists(path, false)) {
+            ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), ErrorInfo.class);
+            return errorInfo;
+        }
+
+        return null;
+    }
+
+    @Override
+    public void setCredentials(String stormId, Credentials creds, Map<String, Object> topoConf) throws NoSuchAlgorithmException {
+        List<ACL> aclList = ClusterUtils.mkTopoOnlyAcls(topoConf);
+        String path = ClusterUtils.credentialsPath(stormId);
+        stateStorage.set_data(path, Utils.serialize(creds), aclList);
+
+    }
+
+    @Override
+    public Credentials credentials(String stormId, Runnable callback) {
+        if (callback != null) {
+            credentialsCallback.put(stormId, callback);
+        }
+        String path = ClusterUtils.credentialsPath(stormId);
+        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, callback != null), Credentials.class);
+
+    }
+
+    @Override
+    public void disconnect() {
+        stateStorage.unregister(stateId);
+        if (solo)
+            stateStorage.close();
+    }
+
+    private List<String> tokenizePath(String path) {
+        String[] toks = path.split("/");
+        java.util.ArrayList<String> rtn = new ArrayList<String>();
+        for (String str : toks) {
+            if (!str.isEmpty()) {
+                rtn.add(str);
+            }
+        }
+        return rtn;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/VersionedData.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/VersionedData.java b/storm-client/src/jvm/org/apache/storm/cluster/VersionedData.java
new file mode 100644
index 0000000..3de2a88
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/VersionedData.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+public class VersionedData<D> {
+    private final int version;
+    private final D data;
+    
+    public VersionedData(int version, D data) {
+        this.version = version;
+        this.data = data;
+    }
+    
+    public int getVersion() {
+        return version;
+    }
+    
+    public D getData() {
+        return data;
+    }
+}


[03/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/grouping/ShuffleGrouping.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/ShuffleGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/ShuffleGrouping.java
new file mode 100644
index 0000000..ad8014c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/ShuffleGrouping.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.grouping;
+
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.task.WorkerTopologyContext;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Random;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class ShuffleGrouping implements CustomStreamGrouping, Serializable {
+    private Random random;
+    private ArrayList<List<Integer>> choices;
+    private AtomicInteger current;
+
+    @Override
+    public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+        random = new Random();
+        choices = new ArrayList<List<Integer>>(targetTasks.size());
+        for (Integer i: targetTasks) {
+            choices.add(Arrays.asList(i));
+        }
+        Collections.shuffle(choices, random);
+        current = new AtomicInteger(0);
+    }
+
+    @Override
+    public List<Integer> chooseTasks(int taskId, List<Object> values) {
+        int rightNow;
+        int size = choices.size();
+        while (true) {
+            rightNow = current.incrementAndGet();
+            if (rightNow < size) {
+                return choices.get(rightNow);
+            } else if (rightNow == size) {
+                current.set(0);
+                return choices.get(0);
+            }
+            //race condition with another thread, and we lost
+            // try again
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/BaseTaskHook.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/BaseTaskHook.java b/storm-client/src/jvm/org/apache/storm/hooks/BaseTaskHook.java
new file mode 100644
index 0000000..3eee8d8
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/BaseTaskHook.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks;
+
+import org.apache.storm.hooks.info.BoltAckInfo;
+import org.apache.storm.hooks.info.BoltExecuteInfo;
+import org.apache.storm.hooks.info.BoltFailInfo;
+import org.apache.storm.hooks.info.EmitInfo;
+import org.apache.storm.hooks.info.SpoutAckInfo;
+import org.apache.storm.hooks.info.SpoutFailInfo;
+import org.apache.storm.task.TopologyContext;
+import java.util.Map;
+
+public class BaseTaskHook implements ITaskHook {
+    @Override
+    public void prepare(Map conf, TopologyContext context) {
+    }
+
+    @Override
+    public void cleanup() {
+    }    
+
+    @Override
+    public void emit(EmitInfo info) {
+    }
+
+    @Override
+    public void spoutAck(SpoutAckInfo info) {
+    }
+
+    @Override
+    public void spoutFail(SpoutFailInfo info) {
+    }
+
+    @Override
+    public void boltAck(BoltAckInfo info) {
+    }
+
+    @Override
+    public void boltFail(BoltFailInfo info) {
+    }
+
+    @Override
+    public void boltExecute(BoltExecuteInfo info) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java b/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java
new file mode 100644
index 0000000..e178280
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks;
+
+import org.apache.storm.task.WorkerTopologyContext;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * A BaseWorkerHook is a noop implementation of IWorkerHook. You
+ * may extends this class and implement any and/or all methods you
+ * need for your workers.
+ */
+public class BaseWorkerHook implements IWorkerHook, Serializable {
+    private static final long serialVersionUID = 2589466485198339529L;
+
+    /**
+     * This method is called when a worker is started
+     *
+     * @param stormConf The Storm configuration for this worker
+     * @param context This object can be used to get information about this worker's place within the topology
+     */
+    @Override
+    public void start(Map stormConf, WorkerTopologyContext context) {
+        // NOOP
+    }
+
+    /**
+     * This method is called right before a worker shuts down
+     */
+    @Override
+    public void shutdown() {
+        // NOOP
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/ITaskHook.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/ITaskHook.java b/storm-client/src/jvm/org/apache/storm/hooks/ITaskHook.java
new file mode 100644
index 0000000..519af0c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/ITaskHook.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks;
+
+import org.apache.storm.hooks.info.BoltAckInfo;
+import org.apache.storm.hooks.info.BoltExecuteInfo;
+import org.apache.storm.hooks.info.SpoutFailInfo;
+import org.apache.storm.hooks.info.SpoutAckInfo;
+import org.apache.storm.hooks.info.EmitInfo;
+import org.apache.storm.hooks.info.BoltFailInfo;
+import org.apache.storm.task.TopologyContext;
+import java.util.Map;
+
+public interface ITaskHook {
+    void prepare(Map conf, TopologyContext context);
+    void cleanup();
+    void emit(EmitInfo info);
+    void spoutAck(SpoutAckInfo info);
+    void spoutFail(SpoutFailInfo info);
+    void boltExecute(BoltExecuteInfo info);
+    void boltAck(BoltAckInfo info);
+    void boltFail(BoltFailInfo info);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java b/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java
new file mode 100644
index 0000000..5f49fe3
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks;
+
+import org.apache.storm.task.WorkerTopologyContext;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * An IWorkerHook represents a topology component that can be executed
+ * when a worker starts, and when a worker shuts down. It can be useful
+ * when you want to execute operations before topology processing starts,
+ * or cleanup operations before your workers shut down.
+ */
+public interface IWorkerHook extends Serializable {
+    /**
+     * This method is called when a worker is started
+     *
+     * @param stormConf The Storm configuration for this worker
+     * @param context This object can be used to get information about this worker's place within the topology
+     */
+    void start(Map stormConf, WorkerTopologyContext context);
+
+    /**
+     * This method is called right before a worker shuts down
+     */
+    void shutdown();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/SubmitterHookException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/SubmitterHookException.java b/storm-client/src/jvm/org/apache/storm/hooks/SubmitterHookException.java
new file mode 100644
index 0000000..b7679a7
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/SubmitterHookException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.storm.hooks;
+
+/**
+ * This Exception is thrown when registered {@link org.apache.storm.ISubmitterHook} could not be initialized or invoked.
+ */
+public class SubmitterHookException extends RuntimeException {
+
+    public SubmitterHookException() {
+    }
+
+    public SubmitterHookException(String message) {
+        super(message);
+    }
+
+    public SubmitterHookException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public SubmitterHookException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/info/BoltAckInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/info/BoltAckInfo.java b/storm-client/src/jvm/org/apache/storm/hooks/info/BoltAckInfo.java
new file mode 100644
index 0000000..e6f4b11
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/info/BoltAckInfo.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks.info;
+
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Tuple;
+
+public class BoltAckInfo {
+    public Tuple tuple;
+    public int ackingTaskId;
+    public Long processLatencyMs; // null if it wasn't sampled
+    
+    public BoltAckInfo(Tuple tuple, int ackingTaskId, Long processLatencyMs) {
+        this.tuple = tuple;
+        this.ackingTaskId = ackingTaskId;
+        this.processLatencyMs = processLatencyMs;
+    }
+
+    public void applyOn(TopologyContext topologyContext) {
+        for (ITaskHook hook : topologyContext.getHooks()) {
+            hook.boltAck(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/info/BoltExecuteInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/info/BoltExecuteInfo.java b/storm-client/src/jvm/org/apache/storm/hooks/info/BoltExecuteInfo.java
new file mode 100644
index 0000000..73a7f33
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/info/BoltExecuteInfo.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks.info;
+
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Tuple;
+
+public class BoltExecuteInfo {
+    public Tuple tuple;
+    public int executingTaskId;
+    public Long executeLatencyMs; // null if it wasn't sampled
+    
+    public BoltExecuteInfo(Tuple tuple, int executingTaskId, Long executeLatencyMs) {
+        this.tuple = tuple;
+        this.executingTaskId = executingTaskId;
+        this.executeLatencyMs = executeLatencyMs;
+    }
+
+    public void applyOn(TopologyContext topologyContext) {
+        for (ITaskHook hook : topologyContext.getHooks()) {
+            hook.boltExecute(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/info/BoltFailInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/info/BoltFailInfo.java b/storm-client/src/jvm/org/apache/storm/hooks/info/BoltFailInfo.java
new file mode 100644
index 0000000..4e1e32d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/info/BoltFailInfo.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks.info;
+
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Tuple;
+
+public class BoltFailInfo {
+    public Tuple tuple;
+    public int failingTaskId;
+    public Long failLatencyMs; // null if it wasn't sampled
+    
+    public BoltFailInfo(Tuple tuple, int failingTaskId, Long failLatencyMs) {
+        this.tuple = tuple;
+        this.failingTaskId = failingTaskId;
+        this.failLatencyMs = failLatencyMs;
+    }
+
+    public void applyOn(TopologyContext topologyContext) {
+        for (ITaskHook hook : topologyContext.getHooks()) {
+            hook.boltFail(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/info/EmitInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/info/EmitInfo.java b/storm-client/src/jvm/org/apache/storm/hooks/info/EmitInfo.java
new file mode 100644
index 0000000..52965a1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/info/EmitInfo.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks.info;
+
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.task.TopologyContext;
+
+import java.util.Collection;
+import java.util.List;
+
+public class EmitInfo {
+    public List<Object> values;
+    public String stream;
+    public int taskId;
+    public Collection<Integer> outTasks;
+    
+    public EmitInfo(List<Object> values, String stream, int taskId, Collection<Integer> outTasks) {
+        this.values = values;
+        this.stream = stream;
+        this.taskId = taskId;
+        this.outTasks = outTasks;
+    }
+
+    public void applyOn(TopologyContext topologyContext) {
+        for (ITaskHook hook : topologyContext.getHooks()) {
+            hook.emit(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutAckInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutAckInfo.java b/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutAckInfo.java
new file mode 100644
index 0000000..4949f0f
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutAckInfo.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks.info;
+
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.task.TopologyContext;
+
+public class SpoutAckInfo {
+    public Object messageId;
+    public int spoutTaskId;
+    public Long completeLatencyMs; // null if it wasn't sampled
+    
+    public SpoutAckInfo(Object messageId, int spoutTaskId, Long completeLatencyMs) {
+        this.messageId = messageId;
+        this.spoutTaskId = spoutTaskId;
+        this.completeLatencyMs = completeLatencyMs;
+    }
+
+    public void applyOn(TopologyContext topologyContext) {
+        for (ITaskHook hook : topologyContext.getHooks()) {
+            hook.spoutAck(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutFailInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutFailInfo.java b/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutFailInfo.java
new file mode 100644
index 0000000..5b40005
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/hooks/info/SpoutFailInfo.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.hooks.info;
+
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.task.TopologyContext;
+
+public class SpoutFailInfo {
+    public Object messageId;
+    public int spoutTaskId;
+    public Long failLatencyMs; // null if it wasn't sampled
+    
+    public SpoutFailInfo(Object messageId, int spoutTaskId, Long failLatencyMs) {
+        this.messageId = messageId;
+        this.spoutTaskId = spoutTaskId;
+        this.failLatencyMs = failLatencyMs;
+    }
+
+    public void applyOn(TopologyContext topologyContext) {
+        for (ITaskHook hook : topologyContext.getHooks()) {
+            hook.spoutFail(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java b/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java
new file mode 100644
index 0000000..ba547ac
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging;
+
+public abstract class ConnectionWithStatus implements IConnection {
+
+  public static enum Status {
+
+    /**
+     * we are establishing a active connection with target host. The new data
+     * sending request can be buffered for future sending, or dropped(cases like
+     * there is no enough memory). It varies with difference IConnection
+     * implementations.
+     */
+    Connecting,
+
+    /**
+     * We have a alive connection channel, which can be used to transfer data.
+     */
+    Ready,
+
+    /**
+     * The connection channel is closed or being closed. We don't accept further
+     * data sending or receiving. All data sending request will be dropped.
+     */
+    Closed
+  }
+
+    /**
+   * whether this connection is available to transfer data
+   */
+  public abstract Status status();
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java b/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java
new file mode 100644
index 0000000..8f415c2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging;
+
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.task.GeneralTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.serialization.KryoTupleDeserializer;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class that is called when a TaskMessage arrives.
+ */
+public class DeserializingConnectionCallback implements IConnectionCallback {
+    private final WorkerState.ILocalTransferCallback _cb;
+    private final Map _conf;
+    private final GeneralTopologyContext _context;
+    private final ThreadLocal<KryoTupleDeserializer> _des =
+         new ThreadLocal<KryoTupleDeserializer>() {
+             @Override
+             protected KryoTupleDeserializer initialValue() {
+                 return new KryoTupleDeserializer(_conf, _context);
+             }
+         };
+
+    public DeserializingConnectionCallback(final Map conf, final GeneralTopologyContext context, WorkerState.ILocalTransferCallback callback) {
+        _conf = conf;
+        _context = context;
+        _cb = callback;
+    }
+
+    @Override
+    public void recv(List<TaskMessage> batch) {
+        KryoTupleDeserializer des = _des.get();
+        ArrayList<AddressedTuple> ret = new ArrayList<>(batch.size());
+        for (TaskMessage message: batch) {
+            ret.add(new AddressedTuple(message.task(), des.deserialize(message.message())));
+        }
+        _cb.transfer(ret);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java b/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java
new file mode 100644
index 0000000..7042dc3
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging;
+
+import org.apache.storm.grouping.Load;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+
+public interface IConnection {
+    /**
+     * Register a callback to be notified when data is ready to be processed.
+     * @param cb the callback to process the messages.
+     */
+    public void registerRecv(IConnectionCallback cb);
+
+    /**
+     * Send load metrics to all downstream connections.
+     * @param taskToLoad a map from the task id to the load for that task.
+     */
+    public void sendLoadMetrics(Map<Integer, Double> taskToLoad);
+    
+    /**
+     * send a message with taskId and payload
+     * @param taskId task ID
+     * @param payload
+     */
+    public void send(int taskId,  byte[] payload);
+    
+    /**
+     * send batch messages
+     * @param msgs
+     */
+
+    public void send(Iterator<TaskMessage> msgs);
+    
+    /**
+     * Get the current load for the given tasks
+     * @param tasks the tasks to look for.
+     * @return a Load for each of the tasks it knows about.
+     */
+    public Map<Integer, Load> getLoad(Collection<Integer> tasks);
+
+    /**
+     * close this connection
+     */
+    public void close();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java b/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java
new file mode 100644
index 0000000..3224f2e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging;
+
+import java.util.List;
+
+/**
+ * A class that is called when a TaskMessage arrives.
+ */
+public interface IConnectionCallback {
+    /**
+     * A batch of new messages have arrived to be processed
+     * @param batch the messages to be processed
+     */
+    public void recv(List<TaskMessage> batch);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/IContext.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/IContext.java b/storm-client/src/jvm/org/apache/storm/messaging/IContext.java
new file mode 100644
index 0000000..72812b1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/IContext.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging;
+
+import java.util.Map;
+
+/**
+ * This interface needs to be implemented for messaging plugin. 
+ * 
+ * Messaging plugin is specified via Storm config parameter, storm.messaging.transport.
+ * 
+ * A messaging plugin should have a default constructor and implements IContext interface.
+ * Upon construction, we will invoke IContext::prepare(storm_conf) to enable context to be configured
+ * according to storm configuration. 
+ */
+public interface IContext {
+    /**
+     * This method is invoked at the startup of messaging plugin
+     * @param storm_conf storm configuration
+     */
+    public void prepare(Map storm_conf);
+    
+    /**
+     * This method is invoked when a worker is unload a messaging plugin
+     */
+    public void term();
+
+    /**
+     * This method establishes a server side connection 
+     * @param storm_id topology ID
+     * @param port port #
+     * @return server side connection
+     */
+    public IConnection bind(String storm_id, int port);
+    
+    /**
+     * This method establish a client side connection to a remote server
+     * @param storm_id topology ID
+     * @param host remote host
+     * @param port remote port
+     * @return client side connection
+     */
+    public IConnection connect(String storm_id, String host, int port);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java b/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java
new file mode 100644
index 0000000..98aba31
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging;
+
+import java.nio.ByteBuffer;
+
+public class TaskMessage {
+    private int _task;
+    private byte[] _message;
+    
+    public TaskMessage(int task, byte[] message) {
+        _task = task;
+        _message = message;
+    }
+    
+    public int task() {
+        return _task;
+    }
+
+    public byte[] message() {
+        return _message;
+    }
+    
+    public ByteBuffer serialize() {
+        ByteBuffer bb = ByteBuffer.allocate(_message.length+2);
+        bb.putShort((short)_task);
+        bb.put(_message);
+        return bb;
+    }
+    
+    public void deserialize(ByteBuffer packet) {
+        if (packet==null) return;
+        _task = packet.getShort();
+        _message = new byte[packet.limit()-2];
+        packet.get(_message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java b/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java
new file mode 100644
index 0000000..511257d
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging;
+
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.lang.reflect.Method;
+import org.apache.storm.Config;
+
+public class TransportFactory {
+    public static final Logger LOG = LoggerFactory.getLogger(TransportFactory.class);
+
+    public static IContext makeContext(Map storm_conf) {
+
+        //get factory class name
+        String transport_plugin_klassName = (String)storm_conf.get(Config.STORM_MESSAGING_TRANSPORT);
+        LOG.info("Storm peer transport plugin:"+transport_plugin_klassName);
+
+        IContext transport;
+        try {
+            //create a factory class
+            Class klass = Class.forName(transport_plugin_klassName);
+            //obtain a context object
+            Object obj = klass.newInstance();
+            if (obj instanceof IContext) {
+                //case 1: plugin is a IContext class
+                transport = (IContext)obj;
+                //initialize with storm configuration
+                transport.prepare(storm_conf);
+            } else {
+                //case 2: Non-IContext plugin must have a makeContext(storm_conf) method that returns IContext object
+                Method method = klass.getMethod("makeContext", Map.class);
+                LOG.debug("object:"+obj+" method:"+method);
+                transport = (IContext) method.invoke(obj, storm_conf);
+            }
+        } catch(Exception e) {
+            throw new RuntimeException("Fail to construct messaging plugin from plugin "+transport_plugin_klassName, e);
+        } 
+        return transport;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java b/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java
new file mode 100644
index 0000000..7300847
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.local;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import org.apache.storm.grouping.Load;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.messaging.IConnectionCallback;
+import org.apache.storm.messaging.IContext;
+
+public class Context implements IContext {
+    private static final Logger LOG = LoggerFactory.getLogger(Context.class);
+
+    private static class LocalServer implements IConnection {
+        volatile IConnectionCallback _cb;
+        final ConcurrentHashMap<Integer, Double> _load = new ConcurrentHashMap<>();
+
+        @Override
+        public void registerRecv(IConnectionCallback cb) {
+            _cb = cb;
+        }
+
+        @Override
+        public void send(int taskId,  byte[] payload) {
+            throw new IllegalArgumentException("SHOULD NOT HAPPEN");
+        }
+ 
+        @Override
+        public void send(Iterator<TaskMessage> msgs) {
+            throw new IllegalArgumentException("SHOULD NOT HAPPEN");
+        }
+
+        @Override
+        public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
+            Map<Integer, Load> ret = new HashMap<>();
+            for (Integer task : tasks) {
+                Double found = _load.get(task);
+                if (found != null) {
+                    ret.put(task, new Load(true, found, 0));
+                }
+            }
+            return ret; 
+        }
+
+        @Override
+        public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
+            _load.putAll(taskToLoad);
+        }
+ 
+        @Override
+        public void close() {
+            //NOOP
+        }
+    };
+
+    private static class LocalClient implements IConnection {
+        private final LocalServer _server;
+        //Messages sent before the server registered a callback
+        private final LinkedBlockingQueue<TaskMessage> _pendingDueToUnregisteredServer;
+        private final ScheduledExecutorService _pendingFlusher;
+
+        public LocalClient(LocalServer server) {
+            _server = server;
+            _pendingDueToUnregisteredServer = new LinkedBlockingQueue<>();
+            _pendingFlusher = Executors.newScheduledThreadPool(1, new ThreadFactory(){
+                @Override
+                public Thread newThread(Runnable runnable) {
+                    Thread thread = new Thread(runnable);
+                    thread.setName("LocalClientFlusher-" + thread.getId());
+                    thread.setDaemon(true);
+                    return thread;
+                }
+            });
+            _pendingFlusher.scheduleAtFixedRate(new Runnable(){
+                @Override
+                public void run(){
+                    try {
+                        //Ensure messages are flushed even if no more sends are performed
+                        flushPending();
+                    } catch (Throwable t) {
+                        LOG.error("Uncaught throwable in pending message flusher thread, messages may be lost", t);
+                        throw new RuntimeException(t);
+                    }
+                }
+            }, 5, 5, TimeUnit.SECONDS);
+        }
+
+        @Override
+        public void registerRecv(IConnectionCallback cb) {
+            throw new IllegalArgumentException("SHOULD NOT HAPPEN");
+        }
+        
+        private void flushPending(){
+            IConnectionCallback serverCb = _server._cb;
+            if (serverCb != null && !_pendingDueToUnregisteredServer.isEmpty()) {
+                ArrayList<TaskMessage> ret = new ArrayList<>();
+                _pendingDueToUnregisteredServer.drainTo(ret);
+                serverCb.recv(ret);
+            }
+        }
+        
+        @Override
+        public void send(int taskId,  byte[] payload) {
+            TaskMessage message = new TaskMessage(taskId, payload);
+            IConnectionCallback serverCb = _server._cb;
+            if (serverCb != null) {
+                flushPending();
+                serverCb.recv(Arrays.asList(message));
+            } else {
+                _pendingDueToUnregisteredServer.add(message);
+            }
+        }
+ 
+        @Override
+        public void send(Iterator<TaskMessage> msgs) {
+            IConnectionCallback serverCb = _server._cb;
+            if (serverCb != null) {
+                flushPending();
+                ArrayList<TaskMessage> ret = new ArrayList<>();
+                while (msgs.hasNext()) {
+                    ret.add(msgs.next());
+                }
+                serverCb.recv(ret);
+            } else {
+                while(msgs.hasNext()){
+                    _pendingDueToUnregisteredServer.add(msgs.next());
+                }
+            }
+        }
+
+        @Override
+        public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
+            return _server.getLoad(tasks);
+        }
+
+        @Override
+        public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
+            _server.sendLoadMetrics(taskToLoad);
+        }
+ 
+        @Override
+        public void close() {
+            _pendingFlusher.shutdown();
+            try{
+                _pendingFlusher.awaitTermination(5, TimeUnit.SECONDS);
+            } catch (InterruptedException e){
+                throw new RuntimeException("Interrupted while awaiting flusher shutdown", e);
+            }
+        }
+    };
+
+    private static ConcurrentHashMap<String, LocalServer> _registry = new ConcurrentHashMap<>();
+    private static LocalServer getLocalServer(String nodeId, int port) {
+        String key = nodeId + "-" + port;
+        LocalServer ret = _registry.get(key);
+        if (ret == null) {
+            ret = new LocalServer();
+            LocalServer tmp = _registry.putIfAbsent(key, ret);
+            if (tmp != null) {
+                ret = tmp;
+            }
+        }
+        return ret;
+    }
+        
+    @SuppressWarnings("rawtypes")
+    @Override
+    public void prepare(Map storm_conf) {
+        //NOOP
+    }
+
+    @Override
+    public IConnection bind(String storm_id, int port) {
+        return getLocalServer(storm_id, port);
+    }
+
+    @Override
+    public IConnection connect(String storm_id, String host, int port) {
+        return new LocalClient(getLocalServer(storm_id, port));
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public void term() {
+        //NOOP
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java
new file mode 100644
index 0000000..8463af6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java
@@ -0,0 +1,612 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Iterator;
+import java.util.Collection;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Timer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.lang.InterruptedException;
+
+import org.apache.storm.Config;
+import org.apache.storm.grouping.Load;
+import org.apache.storm.messaging.ConnectionWithStatus;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.messaging.IConnectionCallback;
+import org.apache.storm.metric.api.IStatefulObject;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.StormBoundedExponentialBackoffRetry;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * A Netty client for sending task messages to a remote destination (Netty server).
+ *
+ * Implementation details:
+ *
+ * - Sending messages, i.e. writing to the channel, is performed asynchronously.
+ * - Messages are sent in batches to optimize for network throughput at the expense of network latency.  The message
+ *   batch size is configurable.
+ * - Connecting and reconnecting are performed asynchronously.
+ *     - Note: The current implementation drops any messages that are being enqueued for sending if the connection to
+ *       the remote destination is currently unavailable.
+ */
+public class Client extends ConnectionWithStatus implements IStatefulObject, ISaslClient {
+    private static final long PENDING_MESSAGES_FLUSH_TIMEOUT_MS = 600000L;
+    private static final long PENDING_MESSAGES_FLUSH_INTERVAL_MS = 1000L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(Client.class);
+    private static final String PREFIX = "Netty-Client-";
+    private static final long NO_DELAY_MS = 0L;
+    private static final Timer timer = new Timer("Netty-ChannelAlive-Timer", true);
+
+    private final Map stormConf;
+    private final StormBoundedExponentialBackoffRetry retryPolicy;
+    private final ClientBootstrap bootstrap;
+    private final InetSocketAddress dstAddress;
+    protected final String dstAddressPrefixedName;
+    private volatile Map<Integer, Double> serverLoad = null;
+
+    /**
+     * The channel used for all write operations from this client to the remote destination.
+     */
+    private final AtomicReference<Channel> channelRef = new AtomicReference<>();
+
+    /**
+     * Total number of connection attempts.
+     */
+    private final AtomicInteger totalConnectionAttempts = new AtomicInteger(0);
+
+    /**
+     * Number of connection attempts since the last disconnect.
+     */
+    private final AtomicInteger connectionAttempts = new AtomicInteger(0);
+
+    /**
+     * Number of messages successfully sent to the remote destination.
+     */
+    private final AtomicInteger messagesSent = new AtomicInteger(0);
+
+    /**
+     * Number of messages that could not be sent to the remote destination.
+     */
+    private final AtomicInteger messagesLost = new AtomicInteger(0);
+
+    /**
+     * Periodically checks for connected channel in order to avoid loss
+     * of messages
+     */
+    private final long CHANNEL_ALIVE_INTERVAL_MS = 30000L;
+
+    /**
+     * Number of messages buffered in memory.
+     */
+    private final AtomicLong pendingMessages = new AtomicLong(0);
+
+    /**
+     * Whether the SASL channel is ready.
+     */
+    private final AtomicBoolean saslChannelReady = new AtomicBoolean(false);
+
+    /**
+     * This flag is set to true if and only if a client instance is being closed.
+     */
+    private volatile boolean closing = false;
+
+    private final Context context;
+
+    private final HashedWheelTimer scheduler;
+
+    private final MessageBuffer batcher;
+
+    private final Object writeLock = new Object();
+
+    @SuppressWarnings("rawtypes")
+    Client(Map stormConf, ChannelFactory factory, HashedWheelTimer scheduler, String host, int port, Context context) {
+        this.stormConf = stormConf;
+        closing = false;
+        this.scheduler = scheduler;
+        this.context = context;
+        int bufferSize = ObjectReader.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
+        // if SASL authentication is disabled, saslChannelReady is initialized as true; otherwise false
+        saslChannelReady.set(!ObjectReader.getBoolean(stormConf.get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION), false));
+        LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize);
+        int messageBatchSize = ObjectReader.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144);
+
+        int maxReconnectionAttempts = ObjectReader.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
+        int minWaitMs = ObjectReader.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS));
+        int maxWaitMs = ObjectReader.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS));
+        retryPolicy = new StormBoundedExponentialBackoffRetry(minWaitMs, maxWaitMs, maxReconnectionAttempts);
+
+        // Initiate connection to remote destination
+        bootstrap = createClientBootstrap(factory, bufferSize, stormConf);
+        dstAddress = new InetSocketAddress(host, port);
+        dstAddressPrefixedName = prefixedName(dstAddress);
+        launchChannelAliveThread();
+        scheduleConnect(NO_DELAY_MS);
+        batcher = new MessageBuffer(messageBatchSize);
+    }
+
+    /**
+     * This thread helps us to check for channel connection periodically.
+     * This is performed just to know whether the destination address
+     * is alive or attempts to refresh connections if not alive. This
+     * solution is better than what we have now in case of a bad channel.
+     */
+    private void launchChannelAliveThread() {
+        // netty TimerTask is already defined and hence a fully
+        // qualified name
+        timer.schedule(new java.util.TimerTask() {
+            public void run() {
+                try {
+                    LOG.debug("running timer task, address {}", dstAddress);
+                    if(closing) {
+                        this.cancel();
+                        return;
+                    }
+                    getConnectedChannel();
+                } catch (Exception exp) {
+                    LOG.error("channel connection error {}", exp);
+                }
+            }
+        }, 0, CHANNEL_ALIVE_INTERVAL_MS);
+    }
+
+    private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize, Map stormConf) {
+        ClientBootstrap bootstrap = new ClientBootstrap(factory);
+        bootstrap.setOption("tcpNoDelay", true);
+        bootstrap.setOption("sendBufferSize", bufferSize);
+        bootstrap.setOption("keepAlive", true);
+        bootstrap.setPipelineFactory(new StormClientPipelineFactory(this, stormConf));
+        return bootstrap;
+    }
+
+    private String prefixedName(InetSocketAddress dstAddress) {
+        if (null != dstAddress) {
+            return PREFIX + dstAddress.toString();
+        }
+        return "";
+    }
+
+    /**
+     * Enqueue a task message to be sent to server
+     */
+    private void scheduleConnect(long delayMs) {
+        scheduler.newTimeout(new Connect(dstAddress), delayMs, TimeUnit.MILLISECONDS);
+    }
+
+    private boolean reconnectingAllowed() {
+        return !closing;
+    }
+
+    private boolean connectionEstablished(Channel channel) {
+        // Because we are using TCP (which is a connection-oriented transport unlike UDP), a connection is only fully
+        // established iff the channel is connected.  That is, a TCP-based channel must be in the CONNECTED state before
+        // anything can be read or written to the channel.
+        //
+        // See:
+        // - http://netty.io/3.9/api/org/jboss/netty/channel/ChannelEvent.html
+        // - http://stackoverflow.com/questions/13356622/what-are-the-netty-channel-state-transitions
+        return channel != null && channel.isConnected();
+    }
+
+    /**
+     * Note:  Storm will check via this method whether a worker can be activated safely during the initial startup of a
+     * topology.  The worker will only be activated once all of the its connections are ready.
+     */
+    @Override
+    public Status status() {
+        if (closing) {
+            return Status.Closed;
+        } else if (!connectionEstablished(channelRef.get())) {
+            return Status.Connecting;
+        } else {
+            if (saslChannelReady.get()) {
+                return Status.Ready;
+            } else {
+                return Status.Connecting; // need to wait until sasl channel is also ready
+            }
+        }
+    }
+
+    /**
+     * Receiving messages is not supported by a client.
+     *
+     * @throws java.lang.UnsupportedOperationException whenever this method is being called.
+     */
+    @Override
+    public void registerRecv(IConnectionCallback cb) {
+        throw new UnsupportedOperationException("Client connection should not receive any messages");
+    }
+
+    @Override
+    public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
+        throw new RuntimeException("Client connection should not send load metrics");
+    }
+
+    @Override
+    public void send(int taskId, byte[] payload) {
+        TaskMessage msg = new TaskMessage(taskId, payload);
+        List<TaskMessage> wrapper = new ArrayList<TaskMessage>(1);
+        wrapper.add(msg);
+        send(wrapper.iterator());
+    }
+
+    /**
+     * Enqueue task messages to be sent to the remote destination (cf. `host` and `port`).
+     */
+    @Override
+    public void send(Iterator<TaskMessage> msgs) {
+        if (closing) {
+            int numMessages = iteratorSize(msgs);
+            LOG.error("discarding {} messages because the Netty client to {} is being closed", numMessages,
+                    dstAddressPrefixedName);
+            return;
+        }
+
+        if (!hasMessages(msgs)) {
+            return;
+        }
+
+        Channel channel = getConnectedChannel();
+        if (channel == null) {
+            /*
+             * Connection is unavailable. We will drop pending messages and let at-least-once message replay kick in.
+             *
+             * Another option would be to buffer the messages in memory.  But this option has the risk of causing OOM errors,
+             * especially for topologies that disable message acking because we don't know whether the connection recovery will
+             * succeed  or not, and how long the recovery will take.
+             */
+            dropMessages(msgs);
+            return;
+        }
+
+        synchronized (writeLock) {
+            while (msgs.hasNext()) {
+                TaskMessage message = msgs.next();
+                MessageBatch full = batcher.add(message);
+                if(full != null){
+                    flushMessages(channel, full);
+                }
+            }
+        }
+
+        if(channel.isWritable()){
+            synchronized (writeLock) {
+                // Netty's internal buffer is not full and we still have message left in the buffer.
+                // We should write the unfilled MessageBatch immediately to reduce latency
+                MessageBatch batch = batcher.drain();
+                if(batch != null) {
+                    flushMessages(channel, batch);
+                }
+            }
+        } else {
+            // Channel's buffer is full, meaning that we have time to wait other messages to arrive, and create a bigger
+            // batch. This yields better throughput.
+            // We can rely on `notifyInterestChanged` to push these messages as soon as there is spece in Netty's buffer
+            // because we know `Channel.isWritable` was false after the messages were already in the buffer.
+        }
+    }
+
+    private Channel getConnectedChannel() {
+        Channel channel = channelRef.get();
+        if (connectionEstablished(channel)) {
+            return channel;
+        } else {
+            // Closing the channel and reconnecting should be done before handling the messages.
+            boolean reconnectScheduled = closeChannelAndReconnect(channel);
+            if (reconnectScheduled) {
+                // Log the connection error only once
+                LOG.error("connection to {} is unavailable", dstAddressPrefixedName);
+            }
+            return null;
+        }
+    }
+
+    public InetSocketAddress getDstAddress() {
+        return dstAddress;
+    }
+
+    private boolean hasMessages(Iterator<TaskMessage> msgs) {
+        return msgs != null && msgs.hasNext();
+    }
+
+    private void dropMessages(Iterator<TaskMessage> msgs) {
+        // We consume the iterator by traversing and thus "emptying" it.
+        int msgCount = iteratorSize(msgs);
+        messagesLost.getAndAdd(msgCount);
+    }
+
+    private int iteratorSize(Iterator<TaskMessage> msgs) {
+        int size = 0;
+        if (msgs != null) {
+            while (msgs.hasNext()) {
+                size++;
+                msgs.next();
+            }
+        }
+        return size;
+    }
+
+    /**
+     * Asynchronously writes the message batch to the channel.
+     *
+     * If the write operation fails, then we will close the channel and trigger a reconnect.
+     */
+    private void flushMessages(Channel channel, final MessageBatch batch) {
+        if (null == batch || batch.isEmpty()) {
+            return;
+        }
+
+        final int numMessages = batch.size();
+        LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString());
+        pendingMessages.addAndGet(numMessages);
+
+        ChannelFuture future = channel.write(batch);
+        future.addListener(new ChannelFutureListener() {
+            public void operationComplete(ChannelFuture future) throws Exception {
+                pendingMessages.addAndGet(0 - numMessages);
+                if (future.isSuccess()) {
+                    LOG.debug("sent {} messages to {}", numMessages, dstAddressPrefixedName);
+                    messagesSent.getAndAdd(batch.size());
+                } else {
+                    LOG.error("failed to send {} messages to {}: {}", numMessages, dstAddressPrefixedName,
+                            future.getCause());
+                    closeChannelAndReconnect(future.getChannel());
+                    messagesLost.getAndAdd(numMessages);
+                }
+            }
+
+        });
+    }
+
+    /**
+     * Schedule a reconnect if we closed a non-null channel, and acquired the right to
+     * provide a replacement by successfully setting a null to the channel field
+     * @param channel
+     * @return if the call scheduled a re-connect task
+     */
+    private boolean closeChannelAndReconnect(Channel channel) {
+        if (channel != null) {
+            channel.close();
+            if (channelRef.compareAndSet(channel, null)) {
+                scheduleConnect(NO_DELAY_MS);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Gracefully close this client.
+     */
+    @Override
+    public void close() {
+        if (!closing) {
+            LOG.info("closing Netty Client {}", dstAddressPrefixedName);
+            context.removeClient(dstAddress.getHostName(),dstAddress.getPort());
+            // Set closing to true to prevent any further reconnection attempts.
+            closing = true;
+            waitForPendingMessagesToBeSent();
+            closeChannel();
+        }
+    }
+
+    private void waitForPendingMessagesToBeSent() {
+        LOG.info("waiting up to {} ms to send {} pending messages to {}",
+                PENDING_MESSAGES_FLUSH_TIMEOUT_MS, pendingMessages.get(), dstAddressPrefixedName);
+        long totalPendingMsgs = pendingMessages.get();
+        long startMs = System.currentTimeMillis();
+        while (pendingMessages.get() != 0) {
+            try {
+                long deltaMs = System.currentTimeMillis() - startMs;
+                if (deltaMs > PENDING_MESSAGES_FLUSH_TIMEOUT_MS) {
+                    LOG.error("failed to send all pending messages to {} within timeout, {} of {} messages were not " +
+                            "sent", dstAddressPrefixedName, pendingMessages.get(), totalPendingMsgs);
+                    break;
+                }
+                Thread.sleep(PENDING_MESSAGES_FLUSH_INTERVAL_MS);
+            } catch (InterruptedException e) {
+                break;
+            }
+        }
+
+    }
+
+    private void closeChannel() {
+        Channel channel = channelRef.get();
+        if (channel != null) {
+            channel.close();
+            LOG.debug("channel to {} closed", dstAddressPrefixedName);
+        }
+    }
+
+    void setLoadMetrics(Map<Integer, Double> taskToLoad) {
+        this.serverLoad = taskToLoad;
+    }
+
+    @Override
+    public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
+        Map<Integer, Double> loadCache = serverLoad;
+        Map<Integer, Load> ret = new HashMap<Integer, Load>();
+        if (loadCache != null) {
+            double clientLoad = Math.min(pendingMessages.get(), 1024)/1024.0;
+            for (Integer task : tasks) {
+                Double found = loadCache.get(task);
+                if (found != null) {
+                    ret.put(task, new Load(true, found, clientLoad));
+                }
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public Object getState() {
+        LOG.debug("Getting metrics for client connection to {}", dstAddressPrefixedName);
+        HashMap<String, Object> ret = new HashMap<String, Object>();
+        ret.put("reconnects", totalConnectionAttempts.getAndSet(0));
+        ret.put("sent", messagesSent.getAndSet(0));
+        ret.put("pending", pendingMessages.get());
+        ret.put("lostOnSend", messagesLost.getAndSet(0));
+        ret.put("dest", dstAddress.toString());
+        String src = srcAddressName();
+        if (src != null) {
+            ret.put("src", src);
+        }
+        return ret;
+    }
+
+    public Map getConfig() {
+        return stormConf;
+    }
+
+    /** ISaslClient interface **/
+    public void channelConnected(Channel channel) {
+//        setChannel(channel);
+    }
+
+    public void channelReady() {
+        saslChannelReady.set(true);
+    }
+
+    public String name() {
+        return (String)stormConf.get(Config.TOPOLOGY_NAME);
+    }
+
+    public String secretKey() {
+        return SaslUtils.getSecretKey(stormConf);
+    }
+    /** end **/
+
+    private String srcAddressName() {
+        String name = null;
+        Channel channel = channelRef.get();
+        if (channel != null) {
+            SocketAddress address = channel.getLocalAddress();
+            if (address != null) {
+                name = address.toString();
+            }
+        }
+        return name;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("Netty client for connecting to %s", dstAddressPrefixedName);
+    }
+
+    /**
+     * Called by Netty thread on change in channel interest
+     * @param channel
+     */
+    public void notifyInterestChanged(Channel channel) {
+        if(channel.isWritable()){
+            synchronized (writeLock) {
+                // Channel is writable again, write if there are any messages pending
+                MessageBatch pending = batcher.drain();
+                flushMessages(channel, pending);
+            }
+        }
+    }
+
+    /**
+     * Asynchronously establishes a Netty connection to the remote address
+     * This task runs on a single thread shared among all clients, and thus
+     * should not perform operations that block.
+     */
+    private class Connect implements TimerTask {
+
+        private final InetSocketAddress address;
+
+        public Connect(InetSocketAddress address) {
+            this.address = address;
+        }
+
+        private void reschedule(Throwable t) {
+            String baseMsg = String.format("connection attempt %s to %s failed", connectionAttempts,
+                    dstAddressPrefixedName);
+            String failureMsg = (t == null) ? baseMsg : baseMsg + ": " + t.toString();
+            LOG.error(failureMsg);
+            long nextDelayMs = retryPolicy.getSleepTimeMs(connectionAttempts.get(), 0);
+            scheduleConnect(nextDelayMs);
+        }
+
+
+        @Override
+        public void run(Timeout timeout) throws Exception {
+            if (reconnectingAllowed()) {
+                final int connectionAttempt = connectionAttempts.getAndIncrement();
+                totalConnectionAttempts.getAndIncrement();
+
+                LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt);
+                ChannelFuture future = bootstrap.connect(address);
+                future.addListener(new ChannelFutureListener() {
+                    @Override
+                    public void operationComplete(ChannelFuture future) throws Exception {
+                        // This call returns immediately
+                        Channel newChannel = future.getChannel();
+
+                        if (future.isSuccess() && connectionEstablished(newChannel)) {
+                            boolean setChannel = channelRef.compareAndSet(null, newChannel);
+                            checkState(setChannel);
+                            LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(),
+                                    connectionAttempt);
+                            if (messagesLost.get() > 0) {
+                                LOG.warn("Re-connection to {} was successful but {} messages has been lost so far", address.toString(), messagesLost.get());
+                            }
+                        } else {
+                            Throwable cause = future.getCause();
+                            reschedule(cause);
+                            if (newChannel != null) {
+                                newChannel.close();
+                            }
+                        }
+                    }
+                });
+            } else {
+                close();
+                throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " +
+                        connectionAttempts + " failed attempts. " + messagesLost.get() + " messages were lost");
+
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java
new file mode 100644
index 0000000..0f7c66b
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.apache.storm.utils.ObjectReader;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.storm.Config;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+
+public class Context implements IContext {
+    @SuppressWarnings("rawtypes")
+    private Map storm_conf;
+    private Map<String, IConnection> connections;
+    private NioClientSocketChannelFactory clientChannelFactory;
+    
+    private HashedWheelTimer clientScheduleService;
+
+    /**
+     * initialization per Storm configuration 
+     */
+    @SuppressWarnings("rawtypes")
+    public void prepare(Map storm_conf) {
+        this.storm_conf = storm_conf;
+        connections = new HashMap<>();
+
+        //each context will have a single client channel factory
+        int maxWorkers = ObjectReader.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS));
+		ThreadFactory bossFactory = new NettyRenameThreadFactory("client" + "-boss");
+        ThreadFactory workerFactory = new NettyRenameThreadFactory("client" + "-worker");
+        if (maxWorkers > 0) {
+            clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
+                    Executors.newCachedThreadPool(workerFactory), maxWorkers);
+        } else {
+            clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
+                    Executors.newCachedThreadPool(workerFactory));
+        }
+        
+        clientScheduleService = new HashedWheelTimer(new NettyRenameThreadFactory("client-schedule-service"));
+    }
+
+    /**
+     * establish a server with a binding port
+     */
+    public synchronized IConnection bind(String storm_id, int port) {
+        IConnection server = new Server(storm_conf, port);
+        connections.put(key(storm_id, port), server);
+        return server;
+    }
+
+    /**
+     * establish a connection to a remote server
+     */
+    public synchronized IConnection connect(String storm_id, String host, int port) {
+        IConnection connection = connections.get(key(host,port));
+        if(connection !=null)
+        {
+            return connection;
+        }
+        IConnection client =  new Client(storm_conf, clientChannelFactory, 
+                clientScheduleService, host, port, this);
+        connections.put(key(host, port), client);
+        return client;
+    }
+
+    synchronized void removeClient(String host, int port) {
+        if (connections != null) {
+            connections.remove(key(host, port));
+        }
+    }
+
+    /**
+     * terminate this context
+     */
+    public synchronized void term() {
+        clientScheduleService.stop();
+
+        for (IConnection conn : connections.values()) {
+            conn.close();
+        }
+
+        connections = null;
+
+        //we need to release resources associated with client channel factory
+        clientChannelFactory.releaseExternalResources();
+
+    }
+
+    private String key(String host, int port) {
+        return String.format("%s:%d", host, port);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java
new file mode 100644
index 0000000..3c7aaba
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.io.IOException;
+
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferOutputStream;
+import org.jboss.netty.buffer.ChannelBuffers;
+
+public enum ControlMessage implements INettySerializable {
+    CLOSE_MESSAGE((short)-100),
+    EOB_MESSAGE((short)-201),
+    OK_RESPONSE((short)-200),
+    FAILURE_RESPONSE((short)-400),
+    SASL_TOKEN_MESSAGE_REQUEST((short)-202),
+    SASL_COMPLETE_REQUEST((short)-203);
+
+    private short code;
+
+    //private constructor
+    private ControlMessage(short code) {
+        this.code = code;
+    }
+
+    /**
+     * @param encoded status code
+     * @return a control message per an encoded status code
+     */
+    public static ControlMessage mkMessage(short encoded) {
+        for(ControlMessage cm: ControlMessage.values()) {
+          if(encoded == cm.code) return cm;
+        }
+        return null;
+    }
+
+    public int encodeLength() {
+        return 2; //short
+    }
+    
+    /**
+     * encode the current Control Message into a channel buffer
+     * @throws IOException
+     */
+    public ChannelBuffer buffer() throws IOException {
+        ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encodeLength()));      
+        write(bout);
+        bout.close();
+        return bout.buffer();
+    }
+
+    public static ControlMessage read(byte[] serial) {
+        ChannelBuffer cm_buffer = ChannelBuffers.copiedBuffer(serial);
+        return mkMessage(cm_buffer.getShort(0));
+    }
+    
+    public void write(ChannelBufferOutputStream bout) throws IOException {
+        bout.writeShort(code);        
+    } 
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/INettySerializable.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/INettySerializable.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/INettySerializable.java
new file mode 100644
index 0000000..0a0236f
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/INettySerializable.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import java.io.IOException;
+import org.jboss.netty.buffer.ChannelBuffer;
+
+public interface INettySerializable {
+    ChannelBuffer buffer() throws IOException;
+    int encodeLength();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslClient.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslClient.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslClient.java
new file mode 100644
index 0000000..681c199
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslClient.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import org.apache.storm.Config;
+
+public interface ISaslClient {
+    void channelConnected(Channel channel);
+    void channelReady();
+    String name();
+    String secretKey();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslServer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslServer.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslServer.java
new file mode 100644
index 0000000..997dbeb
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/ISaslServer.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+
+public interface ISaslServer extends IServer {
+    String name();
+    String secretKey();
+    void authenticated(Channel c);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/messaging/netty/IServer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/IServer.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/IServer.java
new file mode 100644
index 0000000..b04d715
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/IServer.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+
+public interface IServer {
+    void channelConnected(Channel c);
+    void received(Object message, String remote, Channel channel) throws InterruptedException;
+    void closeChannel(Channel c);
+}


[05/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java
new file mode 100644
index 0000000..ca424b3
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java
@@ -0,0 +1,605 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class WorkerResources implements org.apache.thrift.TBase<WorkerResources, WorkerResources._Fields>, java.io.Serializable, Cloneable, Comparable<WorkerResources> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerResources");
+
+  private static final org.apache.thrift.protocol.TField MEM_ON_HEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("mem_on_heap", org.apache.thrift.protocol.TType.DOUBLE, (short)1);
+  private static final org.apache.thrift.protocol.TField MEM_OFF_HEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("mem_off_heap", org.apache.thrift.protocol.TType.DOUBLE, (short)2);
+  private static final org.apache.thrift.protocol.TField CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WorkerResourcesStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WorkerResourcesTupleSchemeFactory());
+  }
+
+  private double mem_on_heap; // optional
+  private double mem_off_heap; // optional
+  private double cpu; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MEM_ON_HEAP((short)1, "mem_on_heap"),
+    MEM_OFF_HEAP((short)2, "mem_off_heap"),
+    CPU((short)3, "cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MEM_ON_HEAP
+          return MEM_ON_HEAP;
+        case 2: // MEM_OFF_HEAP
+          return MEM_OFF_HEAP;
+        case 3: // CPU
+          return CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __MEM_ON_HEAP_ISSET_ID = 0;
+  private static final int __MEM_OFF_HEAP_ISSET_ID = 1;
+  private static final int __CPU_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.MEM_ON_HEAP,_Fields.MEM_OFF_HEAP,_Fields.CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MEM_ON_HEAP, new org.apache.thrift.meta_data.FieldMetaData("mem_on_heap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.MEM_OFF_HEAP, new org.apache.thrift.meta_data.FieldMetaData("mem_off_heap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.CPU, new org.apache.thrift.meta_data.FieldMetaData("cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerResources.class, metaDataMap);
+  }
+
+  public WorkerResources() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WorkerResources(WorkerResources other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.mem_on_heap = other.mem_on_heap;
+    this.mem_off_heap = other.mem_off_heap;
+    this.cpu = other.cpu;
+  }
+
+  public WorkerResources deepCopy() {
+    return new WorkerResources(this);
+  }
+
+  @Override
+  public void clear() {
+    set_mem_on_heap_isSet(false);
+    this.mem_on_heap = 0.0;
+    set_mem_off_heap_isSet(false);
+    this.mem_off_heap = 0.0;
+    set_cpu_isSet(false);
+    this.cpu = 0.0;
+  }
+
+  public double get_mem_on_heap() {
+    return this.mem_on_heap;
+  }
+
+  public void set_mem_on_heap(double mem_on_heap) {
+    this.mem_on_heap = mem_on_heap;
+    set_mem_on_heap_isSet(true);
+  }
+
+  public void unset_mem_on_heap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MEM_ON_HEAP_ISSET_ID);
+  }
+
+  /** Returns true if field mem_on_heap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_mem_on_heap() {
+    return EncodingUtils.testBit(__isset_bitfield, __MEM_ON_HEAP_ISSET_ID);
+  }
+
+  public void set_mem_on_heap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MEM_ON_HEAP_ISSET_ID, value);
+  }
+
+  public double get_mem_off_heap() {
+    return this.mem_off_heap;
+  }
+
+  public void set_mem_off_heap(double mem_off_heap) {
+    this.mem_off_heap = mem_off_heap;
+    set_mem_off_heap_isSet(true);
+  }
+
+  public void unset_mem_off_heap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MEM_OFF_HEAP_ISSET_ID);
+  }
+
+  /** Returns true if field mem_off_heap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_mem_off_heap() {
+    return EncodingUtils.testBit(__isset_bitfield, __MEM_OFF_HEAP_ISSET_ID);
+  }
+
+  public void set_mem_off_heap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MEM_OFF_HEAP_ISSET_ID, value);
+  }
+
+  public double get_cpu() {
+    return this.cpu;
+  }
+
+  public void set_cpu(double cpu) {
+    this.cpu = cpu;
+    set_cpu_isSet(true);
+  }
+
+  public void unset_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CPU_ISSET_ID);
+  }
+
+  /** Returns true if field cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __CPU_ISSET_ID);
+  }
+
+  public void set_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MEM_ON_HEAP:
+      if (value == null) {
+        unset_mem_on_heap();
+      } else {
+        set_mem_on_heap((Double)value);
+      }
+      break;
+
+    case MEM_OFF_HEAP:
+      if (value == null) {
+        unset_mem_off_heap();
+      } else {
+        set_mem_off_heap((Double)value);
+      }
+      break;
+
+    case CPU:
+      if (value == null) {
+        unset_cpu();
+      } else {
+        set_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MEM_ON_HEAP:
+      return get_mem_on_heap();
+
+    case MEM_OFF_HEAP:
+      return get_mem_off_heap();
+
+    case CPU:
+      return get_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MEM_ON_HEAP:
+      return is_set_mem_on_heap();
+    case MEM_OFF_HEAP:
+      return is_set_mem_off_heap();
+    case CPU:
+      return is_set_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WorkerResources)
+      return this.equals((WorkerResources)that);
+    return false;
+  }
+
+  public boolean equals(WorkerResources that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_mem_on_heap = true && this.is_set_mem_on_heap();
+    boolean that_present_mem_on_heap = true && that.is_set_mem_on_heap();
+    if (this_present_mem_on_heap || that_present_mem_on_heap) {
+      if (!(this_present_mem_on_heap && that_present_mem_on_heap))
+        return false;
+      if (this.mem_on_heap != that.mem_on_heap)
+        return false;
+    }
+
+    boolean this_present_mem_off_heap = true && this.is_set_mem_off_heap();
+    boolean that_present_mem_off_heap = true && that.is_set_mem_off_heap();
+    if (this_present_mem_off_heap || that_present_mem_off_heap) {
+      if (!(this_present_mem_off_heap && that_present_mem_off_heap))
+        return false;
+      if (this.mem_off_heap != that.mem_off_heap)
+        return false;
+    }
+
+    boolean this_present_cpu = true && this.is_set_cpu();
+    boolean that_present_cpu = true && that.is_set_cpu();
+    if (this_present_cpu || that_present_cpu) {
+      if (!(this_present_cpu && that_present_cpu))
+        return false;
+      if (this.cpu != that.cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_mem_on_heap = true && (is_set_mem_on_heap());
+    list.add(present_mem_on_heap);
+    if (present_mem_on_heap)
+      list.add(mem_on_heap);
+
+    boolean present_mem_off_heap = true && (is_set_mem_off_heap());
+    list.add(present_mem_off_heap);
+    if (present_mem_off_heap)
+      list.add(mem_off_heap);
+
+    boolean present_cpu = true && (is_set_cpu());
+    list.add(present_cpu);
+    if (present_cpu)
+      list.add(cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WorkerResources other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_mem_on_heap()).compareTo(other.is_set_mem_on_heap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_mem_on_heap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mem_on_heap, other.mem_on_heap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_mem_off_heap()).compareTo(other.is_set_mem_off_heap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_mem_off_heap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mem_off_heap, other.mem_off_heap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_cpu()).compareTo(other.is_set_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cpu, other.cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WorkerResources(");
+    boolean first = true;
+
+    if (is_set_mem_on_heap()) {
+      sb.append("mem_on_heap:");
+      sb.append(this.mem_on_heap);
+      first = false;
+    }
+    if (is_set_mem_off_heap()) {
+      if (!first) sb.append(", ");
+      sb.append("mem_off_heap:");
+      sb.append(this.mem_off_heap);
+      first = false;
+    }
+    if (is_set_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("cpu:");
+      sb.append(this.cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WorkerResourcesStandardSchemeFactory implements SchemeFactory {
+    public WorkerResourcesStandardScheme getScheme() {
+      return new WorkerResourcesStandardScheme();
+    }
+  }
+
+  private static class WorkerResourcesStandardScheme extends StandardScheme<WorkerResources> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerResources struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MEM_ON_HEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.mem_on_heap = iprot.readDouble();
+              struct.set_mem_on_heap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // MEM_OFF_HEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.mem_off_heap = iprot.readDouble();
+              struct.set_mem_off_heap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.cpu = iprot.readDouble();
+              struct.set_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerResources struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_mem_on_heap()) {
+        oprot.writeFieldBegin(MEM_ON_HEAP_FIELD_DESC);
+        oprot.writeDouble(struct.mem_on_heap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_mem_off_heap()) {
+        oprot.writeFieldBegin(MEM_OFF_HEAP_FIELD_DESC);
+        oprot.writeDouble(struct.mem_off_heap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_cpu()) {
+        oprot.writeFieldBegin(CPU_FIELD_DESC);
+        oprot.writeDouble(struct.cpu);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WorkerResourcesTupleSchemeFactory implements SchemeFactory {
+    public WorkerResourcesTupleScheme getScheme() {
+      return new WorkerResourcesTupleScheme();
+    }
+  }
+
+  private static class WorkerResourcesTupleScheme extends TupleScheme<WorkerResources> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WorkerResources struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_mem_on_heap()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_mem_off_heap()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_cpu()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_mem_on_heap()) {
+        oprot.writeDouble(struct.mem_on_heap);
+      }
+      if (struct.is_set_mem_off_heap()) {
+        oprot.writeDouble(struct.mem_off_heap);
+      }
+      if (struct.is_set_cpu()) {
+        oprot.writeDouble(struct.cpu);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WorkerResources struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.mem_on_heap = iprot.readDouble();
+        struct.set_mem_on_heap_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.mem_off_heap = iprot.readDouble();
+        struct.set_mem_off_heap_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.cpu = iprot.readDouble();
+        struct.set_cpu_isSet(true);
+      }
+    }
+  }
+
+}
+


[26/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/GlobalStreamId.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/GlobalStreamId.java b/storm-client/src/jvm/org/apache/storm/generated/GlobalStreamId.java
new file mode 100644
index 0000000..677dfd8
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/GlobalStreamId.java
@@ -0,0 +1,507 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class GlobalStreamId implements org.apache.thrift.TBase<GlobalStreamId, GlobalStreamId._Fields>, java.io.Serializable, Cloneable, Comparable<GlobalStreamId> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GlobalStreamId");
+
+  private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("componentId", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField STREAM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("streamId", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GlobalStreamIdStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GlobalStreamIdTupleSchemeFactory());
+  }
+
+  private String componentId; // required
+  private String streamId; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    COMPONENT_ID((short)1, "componentId"),
+    STREAM_ID((short)2, "streamId");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // COMPONENT_ID
+          return COMPONENT_ID;
+        case 2: // STREAM_ID
+          return STREAM_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("componentId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.STREAM_ID, new org.apache.thrift.meta_data.FieldMetaData("streamId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GlobalStreamId.class, metaDataMap);
+  }
+
+  public GlobalStreamId() {
+  }
+
+  public GlobalStreamId(
+    String componentId,
+    String streamId)
+  {
+    this();
+    this.componentId = componentId;
+    this.streamId = streamId;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GlobalStreamId(GlobalStreamId other) {
+    if (other.is_set_componentId()) {
+      this.componentId = other.componentId;
+    }
+    if (other.is_set_streamId()) {
+      this.streamId = other.streamId;
+    }
+  }
+
+  public GlobalStreamId deepCopy() {
+    return new GlobalStreamId(this);
+  }
+
+  @Override
+  public void clear() {
+    this.componentId = null;
+    this.streamId = null;
+  }
+
+  public String get_componentId() {
+    return this.componentId;
+  }
+
+  public void set_componentId(String componentId) {
+    this.componentId = componentId;
+  }
+
+  public void unset_componentId() {
+    this.componentId = null;
+  }
+
+  /** Returns true if field componentId is set (has been assigned a value) and false otherwise */
+  public boolean is_set_componentId() {
+    return this.componentId != null;
+  }
+
+  public void set_componentId_isSet(boolean value) {
+    if (!value) {
+      this.componentId = null;
+    }
+  }
+
+  public String get_streamId() {
+    return this.streamId;
+  }
+
+  public void set_streamId(String streamId) {
+    this.streamId = streamId;
+  }
+
+  public void unset_streamId() {
+    this.streamId = null;
+  }
+
+  /** Returns true if field streamId is set (has been assigned a value) and false otherwise */
+  public boolean is_set_streamId() {
+    return this.streamId != null;
+  }
+
+  public void set_streamId_isSet(boolean value) {
+    if (!value) {
+      this.streamId = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case COMPONENT_ID:
+      if (value == null) {
+        unset_componentId();
+      } else {
+        set_componentId((String)value);
+      }
+      break;
+
+    case STREAM_ID:
+      if (value == null) {
+        unset_streamId();
+      } else {
+        set_streamId((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case COMPONENT_ID:
+      return get_componentId();
+
+    case STREAM_ID:
+      return get_streamId();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case COMPONENT_ID:
+      return is_set_componentId();
+    case STREAM_ID:
+      return is_set_streamId();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GlobalStreamId)
+      return this.equals((GlobalStreamId)that);
+    return false;
+  }
+
+  public boolean equals(GlobalStreamId that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_componentId = true && this.is_set_componentId();
+    boolean that_present_componentId = true && that.is_set_componentId();
+    if (this_present_componentId || that_present_componentId) {
+      if (!(this_present_componentId && that_present_componentId))
+        return false;
+      if (!this.componentId.equals(that.componentId))
+        return false;
+    }
+
+    boolean this_present_streamId = true && this.is_set_streamId();
+    boolean that_present_streamId = true && that.is_set_streamId();
+    if (this_present_streamId || that_present_streamId) {
+      if (!(this_present_streamId && that_present_streamId))
+        return false;
+      if (!this.streamId.equals(that.streamId))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_componentId = true && (is_set_componentId());
+    list.add(present_componentId);
+    if (present_componentId)
+      list.add(componentId);
+
+    boolean present_streamId = true && (is_set_streamId());
+    list.add(present_streamId);
+    if (present_streamId)
+      list.add(streamId);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GlobalStreamId other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_componentId()).compareTo(other.is_set_componentId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_componentId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.componentId, other.componentId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_streamId()).compareTo(other.is_set_streamId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_streamId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streamId, other.streamId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GlobalStreamId(");
+    boolean first = true;
+
+    sb.append("componentId:");
+    if (this.componentId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.componentId);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("streamId:");
+    if (this.streamId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.streamId);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_componentId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'componentId' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_streamId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'streamId' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GlobalStreamIdStandardSchemeFactory implements SchemeFactory {
+    public GlobalStreamIdStandardScheme getScheme() {
+      return new GlobalStreamIdStandardScheme();
+    }
+  }
+
+  private static class GlobalStreamIdStandardScheme extends StandardScheme<GlobalStreamId> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GlobalStreamId struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // COMPONENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.componentId = iprot.readString();
+              struct.set_componentId_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STREAM_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.streamId = iprot.readString();
+              struct.set_streamId_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GlobalStreamId struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.componentId != null) {
+        oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+        oprot.writeString(struct.componentId);
+        oprot.writeFieldEnd();
+      }
+      if (struct.streamId != null) {
+        oprot.writeFieldBegin(STREAM_ID_FIELD_DESC);
+        oprot.writeString(struct.streamId);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GlobalStreamIdTupleSchemeFactory implements SchemeFactory {
+    public GlobalStreamIdTupleScheme getScheme() {
+      return new GlobalStreamIdTupleScheme();
+    }
+  }
+
+  private static class GlobalStreamIdTupleScheme extends TupleScheme<GlobalStreamId> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GlobalStreamId struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.componentId);
+      oprot.writeString(struct.streamId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GlobalStreamId struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.componentId = iprot.readString();
+      struct.set_componentId_isSet(true);
+      struct.streamId = iprot.readString();
+      struct.set_streamId_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/Grouping.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/Grouping.java b/storm-client/src/jvm/org/apache/storm/generated/Grouping.java
new file mode 100644
index 0000000..ccb19bc
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/Grouping.java
@@ -0,0 +1,800 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class Grouping extends org.apache.thrift.TUnion<Grouping, Grouping._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Grouping");
+  private static final org.apache.thrift.protocol.TField FIELDS_FIELD_DESC = new org.apache.thrift.protocol.TField("fields", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField SHUFFLE_FIELD_DESC = new org.apache.thrift.protocol.TField("shuffle", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField ALL_FIELD_DESC = new org.apache.thrift.protocol.TField("all", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField NONE_FIELD_DESC = new org.apache.thrift.protocol.TField("none", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+  private static final org.apache.thrift.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift.protocol.TField("direct", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField CUSTOM_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("custom_object", org.apache.thrift.protocol.TType.STRUCT, (short)6);
+  private static final org.apache.thrift.protocol.TField CUSTOM_SERIALIZED_FIELD_DESC = new org.apache.thrift.protocol.TField("custom_serialized", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField LOCAL_OR_SHUFFLE_FIELD_DESC = new org.apache.thrift.protocol.TField("local_or_shuffle", org.apache.thrift.protocol.TType.STRUCT, (short)8);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    FIELDS((short)1, "fields"),
+    SHUFFLE((short)2, "shuffle"),
+    ALL((short)3, "all"),
+    NONE((short)4, "none"),
+    DIRECT((short)5, "direct"),
+    CUSTOM_OBJECT((short)6, "custom_object"),
+    CUSTOM_SERIALIZED((short)7, "custom_serialized"),
+    LOCAL_OR_SHUFFLE((short)8, "local_or_shuffle");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FIELDS
+          return FIELDS;
+        case 2: // SHUFFLE
+          return SHUFFLE;
+        case 3: // ALL
+          return ALL;
+        case 4: // NONE
+          return NONE;
+        case 5: // DIRECT
+          return DIRECT;
+        case 6: // CUSTOM_OBJECT
+          return CUSTOM_OBJECT;
+        case 7: // CUSTOM_SERIALIZED
+          return CUSTOM_SERIALIZED;
+        case 8: // LOCAL_OR_SHUFFLE
+          return LOCAL_OR_SHUFFLE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FIELDS, new org.apache.thrift.meta_data.FieldMetaData("fields", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.SHUFFLE, new org.apache.thrift.meta_data.FieldMetaData("shuffle", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class)));
+    tmpMap.put(_Fields.ALL, new org.apache.thrift.meta_data.FieldMetaData("all", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class)));
+    tmpMap.put(_Fields.NONE, new org.apache.thrift.meta_data.FieldMetaData("none", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class)));
+    tmpMap.put(_Fields.DIRECT, new org.apache.thrift.meta_data.FieldMetaData("direct", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class)));
+    tmpMap.put(_Fields.CUSTOM_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("custom_object", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, JavaObject.class)));
+    tmpMap.put(_Fields.CUSTOM_SERIALIZED, new org.apache.thrift.meta_data.FieldMetaData("custom_serialized", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.LOCAL_OR_SHUFFLE, new org.apache.thrift.meta_data.FieldMetaData("local_or_shuffle", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Grouping.class, metaDataMap);
+  }
+
+  public Grouping() {
+    super();
+  }
+
+  public Grouping(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public Grouping(Grouping other) {
+    super(other);
+  }
+  public Grouping deepCopy() {
+    return new Grouping(this);
+  }
+
+  public static Grouping fields(List<String> value) {
+    Grouping x = new Grouping();
+    x.set_fields(value);
+    return x;
+  }
+
+  public static Grouping shuffle(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_shuffle(value);
+    return x;
+  }
+
+  public static Grouping all(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_all(value);
+    return x;
+  }
+
+  public static Grouping none(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_none(value);
+    return x;
+  }
+
+  public static Grouping direct(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_direct(value);
+    return x;
+  }
+
+  public static Grouping custom_object(JavaObject value) {
+    Grouping x = new Grouping();
+    x.set_custom_object(value);
+    return x;
+  }
+
+  public static Grouping custom_serialized(ByteBuffer value) {
+    Grouping x = new Grouping();
+    x.set_custom_serialized(value);
+    return x;
+  }
+
+  public static Grouping custom_serialized(byte[] value) {
+    Grouping x = new Grouping();
+    x.set_custom_serialized(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+    return x;
+  }
+
+  public static Grouping local_or_shuffle(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_local_or_shuffle(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case FIELDS:
+        if (value instanceof List) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type List<String> for field 'fields', but got " + value.getClass().getSimpleName());
+      case SHUFFLE:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'shuffle', but got " + value.getClass().getSimpleName());
+      case ALL:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'all', but got " + value.getClass().getSimpleName());
+      case NONE:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'none', but got " + value.getClass().getSimpleName());
+      case DIRECT:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'direct', but got " + value.getClass().getSimpleName());
+      case CUSTOM_OBJECT:
+        if (value instanceof JavaObject) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type JavaObject for field 'custom_object', but got " + value.getClass().getSimpleName());
+      case CUSTOM_SERIALIZED:
+        if (value instanceof ByteBuffer) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type ByteBuffer for field 'custom_serialized', but got " + value.getClass().getSimpleName());
+      case LOCAL_OR_SHUFFLE:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'local_or_shuffle', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case FIELDS:
+          if (field.type == FIELDS_FIELD_DESC.type) {
+            List<String> fields;
+            {
+              org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
+              fields = new ArrayList<String>(_list8.size);
+              String _elem9;
+              for (int _i10 = 0; _i10 < _list8.size; ++_i10)
+              {
+                _elem9 = iprot.readString();
+                fields.add(_elem9);
+              }
+              iprot.readListEnd();
+            }
+            return fields;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SHUFFLE:
+          if (field.type == SHUFFLE_FIELD_DESC.type) {
+            NullStruct shuffle;
+            shuffle = new NullStruct();
+            shuffle.read(iprot);
+            return shuffle;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case ALL:
+          if (field.type == ALL_FIELD_DESC.type) {
+            NullStruct all;
+            all = new NullStruct();
+            all.read(iprot);
+            return all;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case NONE:
+          if (field.type == NONE_FIELD_DESC.type) {
+            NullStruct none;
+            none = new NullStruct();
+            none.read(iprot);
+            return none;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case DIRECT:
+          if (field.type == DIRECT_FIELD_DESC.type) {
+            NullStruct direct;
+            direct = new NullStruct();
+            direct.read(iprot);
+            return direct;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case CUSTOM_OBJECT:
+          if (field.type == CUSTOM_OBJECT_FIELD_DESC.type) {
+            JavaObject custom_object;
+            custom_object = new JavaObject();
+            custom_object.read(iprot);
+            return custom_object;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case CUSTOM_SERIALIZED:
+          if (field.type == CUSTOM_SERIALIZED_FIELD_DESC.type) {
+            ByteBuffer custom_serialized;
+            custom_serialized = iprot.readBinary();
+            return custom_serialized;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case LOCAL_OR_SHUFFLE:
+          if (field.type == LOCAL_OR_SHUFFLE_FIELD_DESC.type) {
+            NullStruct local_or_shuffle;
+            local_or_shuffle = new NullStruct();
+            local_or_shuffle.read(iprot);
+            return local_or_shuffle;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case FIELDS:
+        List<String> fields = (List<String>)value_;
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, fields.size()));
+          for (String _iter11 : fields)
+          {
+            oprot.writeString(_iter11);
+          }
+          oprot.writeListEnd();
+        }
+        return;
+      case SHUFFLE:
+        NullStruct shuffle = (NullStruct)value_;
+        shuffle.write(oprot);
+        return;
+      case ALL:
+        NullStruct all = (NullStruct)value_;
+        all.write(oprot);
+        return;
+      case NONE:
+        NullStruct none = (NullStruct)value_;
+        none.write(oprot);
+        return;
+      case DIRECT:
+        NullStruct direct = (NullStruct)value_;
+        direct.write(oprot);
+        return;
+      case CUSTOM_OBJECT:
+        JavaObject custom_object = (JavaObject)value_;
+        custom_object.write(oprot);
+        return;
+      case CUSTOM_SERIALIZED:
+        ByteBuffer custom_serialized = (ByteBuffer)value_;
+        oprot.writeBinary(custom_serialized);
+        return;
+      case LOCAL_OR_SHUFFLE:
+        NullStruct local_or_shuffle = (NullStruct)value_;
+        local_or_shuffle.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case FIELDS:
+          List<String> fields;
+          {
+            org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
+            fields = new ArrayList<String>(_list12.size);
+            String _elem13;
+            for (int _i14 = 0; _i14 < _list12.size; ++_i14)
+            {
+              _elem13 = iprot.readString();
+              fields.add(_elem13);
+            }
+            iprot.readListEnd();
+          }
+          return fields;
+        case SHUFFLE:
+          NullStruct shuffle;
+          shuffle = new NullStruct();
+          shuffle.read(iprot);
+          return shuffle;
+        case ALL:
+          NullStruct all;
+          all = new NullStruct();
+          all.read(iprot);
+          return all;
+        case NONE:
+          NullStruct none;
+          none = new NullStruct();
+          none.read(iprot);
+          return none;
+        case DIRECT:
+          NullStruct direct;
+          direct = new NullStruct();
+          direct.read(iprot);
+          return direct;
+        case CUSTOM_OBJECT:
+          JavaObject custom_object;
+          custom_object = new JavaObject();
+          custom_object.read(iprot);
+          return custom_object;
+        case CUSTOM_SERIALIZED:
+          ByteBuffer custom_serialized;
+          custom_serialized = iprot.readBinary();
+          return custom_serialized;
+        case LOCAL_OR_SHUFFLE:
+          NullStruct local_or_shuffle;
+          local_or_shuffle = new NullStruct();
+          local_or_shuffle.read(iprot);
+          return local_or_shuffle;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case FIELDS:
+        List<String> fields = (List<String>)value_;
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, fields.size()));
+          for (String _iter15 : fields)
+          {
+            oprot.writeString(_iter15);
+          }
+          oprot.writeListEnd();
+        }
+        return;
+      case SHUFFLE:
+        NullStruct shuffle = (NullStruct)value_;
+        shuffle.write(oprot);
+        return;
+      case ALL:
+        NullStruct all = (NullStruct)value_;
+        all.write(oprot);
+        return;
+      case NONE:
+        NullStruct none = (NullStruct)value_;
+        none.write(oprot);
+        return;
+      case DIRECT:
+        NullStruct direct = (NullStruct)value_;
+        direct.write(oprot);
+        return;
+      case CUSTOM_OBJECT:
+        JavaObject custom_object = (JavaObject)value_;
+        custom_object.write(oprot);
+        return;
+      case CUSTOM_SERIALIZED:
+        ByteBuffer custom_serialized = (ByteBuffer)value_;
+        oprot.writeBinary(custom_serialized);
+        return;
+      case LOCAL_OR_SHUFFLE:
+        NullStruct local_or_shuffle = (NullStruct)value_;
+        local_or_shuffle.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case FIELDS:
+        return FIELDS_FIELD_DESC;
+      case SHUFFLE:
+        return SHUFFLE_FIELD_DESC;
+      case ALL:
+        return ALL_FIELD_DESC;
+      case NONE:
+        return NONE_FIELD_DESC;
+      case DIRECT:
+        return DIRECT_FIELD_DESC;
+      case CUSTOM_OBJECT:
+        return CUSTOM_OBJECT_FIELD_DESC;
+      case CUSTOM_SERIALIZED:
+        return CUSTOM_SERIALIZED_FIELD_DESC;
+      case LOCAL_OR_SHUFFLE:
+        return LOCAL_OR_SHUFFLE_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public List<String> get_fields() {
+    if (getSetField() == _Fields.FIELDS) {
+      return (List<String>)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'fields' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_fields(List<String> value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.FIELDS;
+    value_ = value;
+  }
+
+  public NullStruct get_shuffle() {
+    if (getSetField() == _Fields.SHUFFLE) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'shuffle' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_shuffle(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SHUFFLE;
+    value_ = value;
+  }
+
+  public NullStruct get_all() {
+    if (getSetField() == _Fields.ALL) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'all' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_all(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.ALL;
+    value_ = value;
+  }
+
+  public NullStruct get_none() {
+    if (getSetField() == _Fields.NONE) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'none' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_none(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.NONE;
+    value_ = value;
+  }
+
+  public NullStruct get_direct() {
+    if (getSetField() == _Fields.DIRECT) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'direct' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_direct(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.DIRECT;
+    value_ = value;
+  }
+
+  public JavaObject get_custom_object() {
+    if (getSetField() == _Fields.CUSTOM_OBJECT) {
+      return (JavaObject)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'custom_object' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_custom_object(JavaObject value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.CUSTOM_OBJECT;
+    value_ = value;
+  }
+
+  public byte[] get_custom_serialized() {
+    set_custom_serialized(org.apache.thrift.TBaseHelper.rightSize(buffer_for_custom_serialized()));
+    ByteBuffer b = buffer_for_custom_serialized();
+    return b == null ? null : b.array();
+  }
+
+  public ByteBuffer buffer_for_custom_serialized() {
+    if (getSetField() == _Fields.CUSTOM_SERIALIZED) {
+      return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue());
+    } else {
+      throw new RuntimeException("Cannot get field 'custom_serialized' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_custom_serialized(byte[] value) {
+    set_custom_serialized(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+  }
+
+  public void set_custom_serialized(ByteBuffer value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.CUSTOM_SERIALIZED;
+    value_ = value;
+  }
+
+  public NullStruct get_local_or_shuffle() {
+    if (getSetField() == _Fields.LOCAL_OR_SHUFFLE) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'local_or_shuffle' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_local_or_shuffle(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.LOCAL_OR_SHUFFLE;
+    value_ = value;
+  }
+
+  public boolean is_set_fields() {
+    return setField_ == _Fields.FIELDS;
+  }
+
+
+  public boolean is_set_shuffle() {
+    return setField_ == _Fields.SHUFFLE;
+  }
+
+
+  public boolean is_set_all() {
+    return setField_ == _Fields.ALL;
+  }
+
+
+  public boolean is_set_none() {
+    return setField_ == _Fields.NONE;
+  }
+
+
+  public boolean is_set_direct() {
+    return setField_ == _Fields.DIRECT;
+  }
+
+
+  public boolean is_set_custom_object() {
+    return setField_ == _Fields.CUSTOM_OBJECT;
+  }
+
+
+  public boolean is_set_custom_serialized() {
+    return setField_ == _Fields.CUSTOM_SERIALIZED;
+  }
+
+
+  public boolean is_set_local_or_shuffle() {
+    return setField_ == _Fields.LOCAL_OR_SHUFFLE;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof Grouping) {
+      return equals((Grouping)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(Grouping other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(Grouping other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBAuthorizationException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBAuthorizationException.java b/storm-client/src/jvm/org/apache/storm/generated/HBAuthorizationException.java
new file mode 100644
index 0000000..cb31213
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBAuthorizationException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBAuthorizationException extends TException implements org.apache.thrift.TBase<HBAuthorizationException, HBAuthorizationException._Fields>, java.io.Serializable, Cloneable, Comparable<HBAuthorizationException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBAuthorizationException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBAuthorizationExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBAuthorizationExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBAuthorizationException.class, metaDataMap);
+  }
+
+  public HBAuthorizationException() {
+  }
+
+  public HBAuthorizationException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBAuthorizationException(HBAuthorizationException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public HBAuthorizationException deepCopy() {
+    return new HBAuthorizationException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBAuthorizationException)
+      return this.equals((HBAuthorizationException)that);
+    return false;
+  }
+
+  public boolean equals(HBAuthorizationException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBAuthorizationException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBAuthorizationException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBAuthorizationExceptionStandardSchemeFactory implements SchemeFactory {
+    public HBAuthorizationExceptionStandardScheme getScheme() {
+      return new HBAuthorizationExceptionStandardScheme();
+    }
+  }
+
+  private static class HBAuthorizationExceptionStandardScheme extends StandardScheme<HBAuthorizationException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBAuthorizationExceptionTupleSchemeFactory implements SchemeFactory {
+    public HBAuthorizationExceptionTupleScheme getScheme() {
+      return new HBAuthorizationExceptionTupleScheme();
+    }
+  }
+
+  private static class HBAuthorizationExceptionTupleScheme extends TupleScheme<HBAuthorizationException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/HBExecutionException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBExecutionException.java b/storm-client/src/jvm/org/apache/storm/generated/HBExecutionException.java
new file mode 100644
index 0000000..697dd06
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBExecutionException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBExecutionException extends TException implements org.apache.thrift.TBase<HBExecutionException, HBExecutionException._Fields>, java.io.Serializable, Cloneable, Comparable<HBExecutionException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBExecutionException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBExecutionExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBExecutionExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBExecutionException.class, metaDataMap);
+  }
+
+  public HBExecutionException() {
+  }
+
+  public HBExecutionException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBExecutionException(HBExecutionException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public HBExecutionException deepCopy() {
+    return new HBExecutionException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBExecutionException)
+      return this.equals((HBExecutionException)that);
+    return false;
+  }
+
+  public boolean equals(HBExecutionException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBExecutionException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBExecutionException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBExecutionExceptionStandardSchemeFactory implements SchemeFactory {
+    public HBExecutionExceptionStandardScheme getScheme() {
+      return new HBExecutionExceptionStandardScheme();
+    }
+  }
+
+  private static class HBExecutionExceptionStandardScheme extends StandardScheme<HBExecutionException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBExecutionException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBExecutionException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBExecutionExceptionTupleSchemeFactory implements SchemeFactory {
+    public HBExecutionExceptionTupleScheme getScheme() {
+      return new HBExecutionExceptionTupleScheme();
+    }
+  }
+
+  private static class HBExecutionExceptionTupleScheme extends TupleScheme<HBExecutionException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBExecutionException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBExecutionException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+


[51/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

* break down 'storm-core' into multiple artifacts
  * 'storm-client': topology and worker, play as client SDK
  * 'storm-client-misc': client SDK misc which is not added due to dependency addition
  * 'storm-server': LocalCluster and related daemons
  * 'storm-core': Clojure things and unported daemons (webapp) and tests
  * 'storm-webapp': DRPC HTTP server, Logviewer and UI will be placed here
* Supervisor will set classpath for worker to just ensure 'storm-client', not 'storm-core'
* addresse documentation
* apply the change to all modules's pom.xml
* add new modules into Travis build
* NOTE: we don't shade and relocate what we have been relocated
  * we should discuss how to overcome dependency issue


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

Branch: refs/heads/master
Commit: 4de339a8cd74ac06211d5a8e4f56c2ea7f6ee951
Parents: c652d3f
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Wed Mar 29 08:08:54 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Apr 6 09:24:43 2017 +0900

----------------------------------------------------------------------
 .travis.yml                                     |     4 +-
 DEVELOPER.md                                    |     2 +-
 bin/storm.py                                    |     5 +-
 ...Defining-a-non-jvm-language-dsl-for-storm.md |     2 +-
 docs/Lifecycle-of-a-topology.md                 |     2 +-
 docs/Maven.md                                   |     2 +-
 docs/Metrics.md                                 |    26 +-
 docs/Storm-Scheduler.md                         |     4 +-
 docs/Stream-API.md                              |     2 +-
 docs/Structure-of-the-codebase.md               |    80 +-
 docs/Trident-API-Overview.md                    |     2 +-
 docs/Trident-spouts.md                          |     8 +-
 docs/Trident-state.md                           |     8 +-
 docs/Windowing.md                               |     2 +-
 docs/storm-kinesis.md                           |     2 +-
 examples/storm-elasticsearch-examples/pom.xml   |     2 +-
 examples/storm-hbase-examples/pom.xml           |     2 +-
 examples/storm-hdfs-examples/pom.xml            |     2 +-
 .../storm/hdfs/spout/HdfsSpoutTopology.java     |     2 +-
 examples/storm-hive-examples/pom.xml            |     2 +-
 examples/storm-jdbc-examples/pom.xml            |     2 +-
 examples/storm-jms-examples/pom.xml             |     4 +-
 examples/storm-kafka-client-examples/pom.xml    |     2 +-
 examples/storm-kafka-examples/pom.xml           |     2 +-
 .../storm/kafka/trident/DrpcResultsPrinter.java |     4 +-
 examples/storm-mongodb-examples/pom.xml         |     2 +-
 .../storm/mongodb/topology/WordCounter.java     |     2 -
 examples/storm-mqtt-examples/pom.xml            |     2 +-
 examples/storm-opentsdb-examples/pom.xml        |     2 +-
 examples/storm-perf/pom.xml                     |     2 +-
 .../org/apache/storm/perf/KafkaHdfsTopo.java    |     3 +-
 .../storm/perf/KafkaSpoutNullBoltTopo.java      |     3 +-
 .../storm/perf/utils/BasicMetricsCollector.java |     1 -
 .../org/apache/storm/perf/utils/Helper.java     |     5 +-
 examples/storm-pmml-examples/pom.xml            |     2 +-
 examples/storm-redis-examples/pom.xml           |     2 +-
 .../storm/redis/topology/WordCounter.java       |     2 -
 examples/storm-solr-examples/pom.xml            |     2 +-
 examples/storm-starter/README.markdown          |     2 +-
 examples/storm-starter/pom.xml                  |    13 +-
 .../apache/storm/starter/clj/exclamation.clj    |     6 +-
 .../storm/starter/clj/rolling_top_words.clj     |     8 +-
 .../starter/BlobStoreAPIWordCountTopology.java  |     3 -
 .../storm/starter/FastWordCountTopology.java    |     3 +-
 .../storm/starter/InOrderDeliveryTest.java      |     4 +-
 .../storm/starter/MultipleLoggerTopology.java   |     1 -
 .../apache/storm/starter/StatefulTopology.java  |   143 +
 .../starter/StatefulWindowingTopology.java      |   111 +
 .../storm/starter/ThroughputVsLatency.java      |    10 +-
 .../storm/starter/streams/AggregateExample.java |     8 -
 .../src/jvm/storm/starter/StatefulTopology.java |   144 -
 .../starter/StatefulWindowingTopology.java      |   112 -
 external/storm-cassandra/pom.xml                |     9 +-
 .../storm/cassandra/bolt/BaseCassandraBolt.java |     2 -
 .../storm/cassandra/client/CassandraConf.java   |    15 +-
 external/storm-druid/pom.xml                    |     8 +-
 external/storm-elasticsearch/pom.xml            |     2 +-
 external/storm-eventhubs/pom.xml                |     9 +-
 .../eventhubs/samples/AtMostOnceEventCount.java |    54 -
 .../storm/eventhubs/samples/EventCount.java     |   157 -
 .../storm/eventhubs/samples/EventHubLoop.java   |    52 -
 .../samples/OpaqueTridentEventCount.java        |    53 -
 .../samples/TransactionalTridentEventCount.java |    81 -
 .../eventhubs/samples/bolt/GlobalCountBolt.java |    88 -
 .../samples/bolt/PartialCountBolt.java          |    68 -
 .../eventhubs/samples/AtMostOnceEventCount.java |    54 +
 .../storm/eventhubs/samples/EventCount.java     |   157 +
 .../storm/eventhubs/samples/EventHubLoop.java   |    52 +
 .../samples/OpaqueTridentEventCount.java        |    53 +
 .../samples/TransactionalTridentEventCount.java |    81 +
 .../eventhubs/samples/bolt/GlobalCountBolt.java |    88 +
 .../samples/bolt/PartialCountBolt.java          |    68 +
 external/storm-hbase/pom.xml                    |     2 +-
 external/storm-hdfs/pom.xml                     |     6 +-
 .../storm/hdfs/blobstore/HdfsBlobStoreImpl.java |     4 +-
 .../storm/hdfs/blobstore/BlobStoreTest.java     |     6 -
 external/storm-hive/pom.xml                     |     4 +-
 external/storm-jdbc/pom.xml                     |     2 +-
 external/storm-jms/pom.xml                      |     2 +-
 .../org/apache/storm/jms/spout/JmsSpout.java    |     2 +-
 .../storm/jms/trident/TridentJmsSpout.java      |     2 +-
 external/storm-kafka-client/pom.xml             |     9 +-
 external/storm-kafka/pom.xml                    |     2 +-
 .../storm/kafka/DynamicBrokersReader.java       |    12 +-
 .../src/jvm/org/apache/storm/kafka/ZkState.java |    10 +-
 .../org/apache/storm/kafka/KafkaUtilsTest.java  |     2 +-
 .../test/org/apache/storm/kafka/TestUtils.java  |     2 -
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  |     2 +-
 external/storm-kinesis/README.md                |     2 +-
 external/storm-kinesis/pom.xml                  |     2 +-
 external/storm-metrics/pom.xml                  |     2 +-
 external/storm-mongodb/pom.xml                  |     6 +-
 external/storm-mqtt/pom.xml                     |     8 +-
 external/storm-opentsdb/pom.xml                 |     2 +-
 external/storm-pmml/pom.xml                     |     2 +-
 external/storm-redis/pom.xml                    |     2 +-
 external/storm-solr/pom.xml                     |     2 +-
 flux/README.md                                  |    22 +
 .../apache/storm/flux/examples/WordCounter.java |     1 -
 flux/pom.xml                                    |     9 +-
 integration-test/pom.xml                        |     2 +-
 .../st/tests/window/SlidingWindowTest.java      |     2 +-
 .../apache/storm/st/wrapper/StormCluster.java   |     2 +-
 .../org/apache/storm/st/wrapper/TopoWrap.java   |     2 +-
 pom.xml                                         |     5 +-
 sql/storm-sql-core/pom.xml                      |     8 +-
 sql/storm-sql-external/storm-sql-hdfs/pom.xml   |     2 +-
 sql/storm-sql-external/storm-sql-kafka/pom.xml  |     2 +-
 .../storm-sql-mongodb/pom.xml                   |     2 +-
 sql/storm-sql-external/storm-sql-redis/pom.xml  |     2 +-
 sql/storm-sql-runtime/pom.xml                   |     2 +-
 .../storm/sql/runtime/utils/SerdeUtils.java     |     6 +-
 storm-client-misc/pom.xml                       |    50 +
 .../metric/HttpForwardingMetricsConsumer.java   |    85 +
 .../metric/HttpForwardingMetricsServer.java     |   118 +
 storm-client/pom.xml                            |   336 +
 storm-client/src/genthrift.sh                   |    34 +
 storm-client/src/java_license_header.txt        |    17 +
 .../src/jvm/org/apache/storm/Config.java        |  1697 +
 .../src/jvm/org/apache/storm/Constants.java     |    60 +
 .../org/apache/storm/ICredentialsListener.java  |    32 +
 .../src/jvm/org/apache/storm/ILocalCluster.java |   176 +
 .../src/jvm/org/apache/storm/ILocalDRPC.java    |    36 +
 .../jvm/org/apache/storm/ISubmitterHook.java    |    31 +
 .../src/jvm/org/apache/storm/LogWriter.java     |    83 +
 .../jvm/org/apache/storm/StormSubmitter.java    |   603 +
 .../src/jvm/org/apache/storm/StormTimer.java    |   242 +
 .../src/jvm/org/apache/storm/Thrift.java        |   350 +
 .../storm/annotation/InterfaceStability.java    |    54 +
 .../storm/blobstore/AtomicOutputStream.java     |    32 +
 .../org/apache/storm/blobstore/BlobStore.java   |   490 +
 .../storm/blobstore/BlobStoreAclHandler.java    |   409 +
 .../apache/storm/blobstore/BlobStoreFile.java   |    50 +
 .../apache/storm/blobstore/ClientBlobStore.java |   201 +
 .../storm/blobstore/InputStreamWithMeta.java    |    26 +
 .../org/apache/storm/blobstore/KeyFilter.java   |    22 +
 .../apache/storm/blobstore/NimbusBlobStore.java |   428 +
 .../src/jvm/org/apache/storm/bolt/JoinBolt.java |   595 +
 .../storm/callback/DefaultWatcherCallBack.java  |    35 +
 .../apache/storm/callback/WatcherCallBack.java  |    24 +
 .../storm/callback/ZKStateChangedCallback.java  |    25 +
 .../org/apache/storm/clojure/ClojureBolt.java   |   120 +
 .../org/apache/storm/clojure/ClojureSpout.java  |   153 +
 .../org/apache/storm/clojure/RichShellBolt.java |    51 +
 .../apache/storm/clojure/RichShellSpout.java    |    51 +
 .../storm/cluster/ClusterStateContext.java      |    41 +
 .../storm/cluster/ClusterStateListener.java     |    22 +
 .../org/apache/storm/cluster/ClusterUtils.java  |   244 +
 .../apache/storm/cluster/ConnectionState.java   |    24 +
 .../org/apache/storm/cluster/DaemonType.java    |    27 +
 .../org/apache/storm/cluster/ExecutorBeat.java  |    44 +
 .../org/apache/storm/cluster/IStateStorage.java |   219 +
 .../storm/cluster/IStormClusterState.java       |   198 +
 .../storm/cluster/PaceMakerStateStorage.java    |   265 +
 .../cluster/PaceMakerStateStorageFactory.java   |    37 +
 .../storm/cluster/StateStorageFactory.java      |    28 +
 .../storm/cluster/StormClusterStateImpl.java    |   735 +
 .../org/apache/storm/cluster/VersionedData.java |    36 +
 .../apache/storm/cluster/ZKStateStorage.java    |   252 +
 .../storm/cluster/ZKStateStorageFactory.java    |    36 +
 .../storm/container/cgroup/CgroupCenter.java    |   216 +
 .../storm/container/cgroup/CgroupCommon.java    |   270 +
 .../container/cgroup/CgroupCommonOperation.java |    81 +
 .../container/cgroup/CgroupCoreFactory.java     |    72 +
 .../storm/container/cgroup/CgroupOperation.java |    79 +
 .../storm/container/cgroup/CgroupUtils.java     |   118 +
 .../apache/storm/container/cgroup/Device.java   |    75 +
 .../storm/container/cgroup/Hierarchy.java       |   130 +
 .../storm/container/cgroup/SubSystem.java       |    81 +
 .../storm/container/cgroup/SubSystemType.java   |    36 +
 .../storm/container/cgroup/SystemOperation.java |    76 +
 .../storm/container/cgroup/core/BlkioCore.java  |   213 +
 .../storm/container/cgroup/core/CgroupCore.java |    26 +
 .../storm/container/cgroup/core/CpuCore.java    |   135 +
 .../container/cgroup/core/CpuacctCore.java      |    71 +
 .../storm/container/cgroup/core/CpusetCore.java |   209 +
 .../container/cgroup/core/DevicesCore.java      |   189 +
 .../container/cgroup/core/FreezerCore.java      |    66 +
 .../storm/container/cgroup/core/MemoryCore.java |   188 +
 .../storm/container/cgroup/core/NetClsCore.java |    69 +
 .../container/cgroup/core/NetPrioCore.java      |    65 +
 .../storm/coordination/BatchBoltExecutor.java   |   109 +
 .../coordination/BatchOutputCollector.java      |    46 +
 .../coordination/BatchOutputCollectorImpl.java  |    53 +
 .../coordination/BatchSubtopologyBuilder.java   |   447 +
 .../storm/coordination/CoordinatedBolt.java     |   385 +
 .../apache/storm/coordination/IBatchBolt.java   |    30 +
 .../src/jvm/org/apache/storm/daemon/Acker.java  |   139 +
 .../org/apache/storm/daemon/DaemonCommon.java   |    22 +
 .../org/apache/storm/daemon/GrouperFactory.java |   243 +
 .../org/apache/storm/daemon/Shutdownable.java   |    22 +
 .../org/apache/storm/daemon/StormCommon.java    |   594 +
 .../src/jvm/org/apache/storm/daemon/Task.java   |   247 +
 .../daemon/metrics/BuiltinBoltMetrics.java      |    78 +
 .../storm/daemon/metrics/BuiltinMetrics.java    |    33 +
 .../daemon/metrics/BuiltinMetricsUtil.java      |    81 +
 .../daemon/metrics/BuiltinSpoutMetrics.java     |    64 +
 .../daemon/metrics/SpoutThrottlingMetrics.java  |    57 +
 .../storm/daemon/supervisor/AdvancedFSOps.java  |   367 +
 .../supervisor/ClientSupervisorUtils.java       |   167 +
 .../daemon/supervisor/ExitCodeCallback.java     |    30 +
 .../storm/daemon/supervisor/IAdvancedFSOps.java |   173 +
 .../storm/daemon/worker/LogConfigManager.java   |   156 +
 .../org/apache/storm/daemon/worker/Worker.java  |   459 +
 .../apache/storm/daemon/worker/WorkerState.java |   690 +
 .../dependency/DependencyBlobStoreUtils.java    |    43 +
 .../dependency/DependencyPropertiesParser.java  |    61 +
 .../storm/dependency/DependencyUploader.java    |   167 +
 .../dependency/FileNotAvailableException.java   |    33 +
 .../storm/drpc/DRPCInvocationsClient.java       |   130 +
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |   287 +
 .../jvm/org/apache/storm/drpc/JoinResult.java   |    75 +
 .../org/apache/storm/drpc/KeyedFairBolt.java    |    93 +
 .../storm/drpc/LinearDRPCInputDeclarer.java     |    52 +
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |   393 +
 .../org/apache/storm/drpc/PrepareRequest.java   |    60 +
 .../org/apache/storm/drpc/ReturnResults.java    |   138 +
 .../jvm/org/apache/storm/executor/Executor.java |   583 +
 .../apache/storm/executor/ExecutorShutdown.java |   114 +
 .../apache/storm/executor/ExecutorTransfer.java |    88 +
 .../apache/storm/executor/IRunningExecutor.java |    31 +
 .../apache/storm/executor/LocalExecutor.java    |    56 +
 .../org/apache/storm/executor/TupleInfo.java    |    90 +
 .../storm/executor/bolt/BoltExecutor.java       |   139 +
 .../executor/bolt/BoltOutputCollectorImpl.java  |   171 +
 .../storm/executor/error/IReportError.java      |    22 +
 .../storm/executor/error/ReportError.java       |    77 +
 .../storm/executor/error/ReportErrorAndDie.java |    49 +
 .../storm/executor/spout/SpoutExecutor.java     |   259 +
 .../spout/SpoutOutputCollectorImpl.java         |   147 +
 .../apache/storm/generated/AccessControl.java   |   627 +
 .../storm/generated/AccessControlType.java      |    62 +
 .../storm/generated/AlreadyAliveException.java  |   406 +
 .../org/apache/storm/generated/Assignment.java  |  1159 +
 .../storm/generated/AuthorizationException.java |   406 +
 .../storm/generated/BeginDownloadResult.java    |   608 +
 .../jvm/org/apache/storm/generated/Bolt.java    |   514 +
 .../storm/generated/BoltAggregateStats.java     |   704 +
 .../org/apache/storm/generated/BoltStats.java   |  1390 +
 .../apache/storm/generated/ClusterSummary.java  |   879 +
 .../storm/generated/ClusterWorkerHeartbeat.java |   768 +
 .../storm/generated/CommonAggregateStats.java   |  1060 +
 .../generated/ComponentAggregateStats.java      |   752 +
 .../apache/storm/generated/ComponentCommon.java |   852 +
 .../apache/storm/generated/ComponentObject.java |   462 +
 .../storm/generated/ComponentPageInfo.java      |  2352 +
 .../apache/storm/generated/ComponentType.java   |    62 +
 .../org/apache/storm/generated/Credentials.java |   458 +
 .../storm/generated/DRPCExceptionType.java      |    68 +
 .../storm/generated/DRPCExecutionException.java |   528 +
 .../org/apache/storm/generated/DRPCRequest.java |   507 +
 .../apache/storm/generated/DebugOptions.java    |   506 +
 .../apache/storm/generated/DistributedRPC.java  |  1328 +
 .../generated/DistributedRPCInvocations.java    |  3904 ++
 .../org/apache/storm/generated/ErrorInfo.java   |   714 +
 .../storm/generated/ExecutorAggregateStats.java |   526 +
 .../apache/storm/generated/ExecutorInfo.java    |   499 +
 .../storm/generated/ExecutorSpecificStats.java  |   387 +
 .../apache/storm/generated/ExecutorStats.java   |   915 +
 .../apache/storm/generated/ExecutorSummary.java |   922 +
 .../apache/storm/generated/GetInfoOptions.java  |   422 +
 .../apache/storm/generated/GlobalStreamId.java  |   507 +
 .../org/apache/storm/generated/Grouping.java    |   800 +
 .../generated/HBAuthorizationException.java     |   406 +
 .../storm/generated/HBExecutionException.java   |   406 +
 .../org/apache/storm/generated/HBMessage.java   |   636 +
 .../apache/storm/generated/HBMessageData.java   |   640 +
 .../jvm/org/apache/storm/generated/HBNodes.java |   461 +
 .../jvm/org/apache/storm/generated/HBPulse.java |   522 +
 .../org/apache/storm/generated/HBRecords.java   |   466 +
 .../storm/generated/HBServerMessageType.java    |   113 +
 .../generated/InvalidTopologyException.java     |   406 +
 .../org/apache/storm/generated/JavaObject.java  |   561 +
 .../apache/storm/generated/JavaObjectArg.java   |   631 +
 .../generated/KeyAlreadyExistsException.java    |   406 +
 .../storm/generated/KeyNotFoundException.java   |   406 +
 .../org/apache/storm/generated/KillOptions.java |   407 +
 .../storm/generated/LSApprovedWorkers.java      |   458 +
 .../generated/LSSupervisorAssignments.java      |   471 +
 .../apache/storm/generated/LSSupervisorId.java  |   406 +
 .../apache/storm/generated/LSTopoHistory.java   |   805 +
 .../storm/generated/LSTopoHistoryList.java      |   460 +
 .../storm/generated/LSWorkerHeartbeat.java      |   755 +
 .../apache/storm/generated/ListBlobsResult.java |   556 +
 .../apache/storm/generated/LocalAssignment.java |   676 +
 .../apache/storm/generated/LocalStateData.java  |   471 +
 .../org/apache/storm/generated/LogConfig.java   |   475 +
 .../org/apache/storm/generated/LogLevel.java    |   836 +
 .../apache/storm/generated/LogLevelAction.java  |    65 +
 .../jvm/org/apache/storm/generated/Nimbus.java  | 47241 +++++++++++++++++
 .../apache/storm/generated/NimbusSummary.java   |   796 +
 .../org/apache/storm/generated/NodeInfo.java    |   556 +
 .../storm/generated/NotAliveException.java      |   406 +
 .../org/apache/storm/generated/NullStruct.java  |   300 +
 .../apache/storm/generated/NumErrorsChoice.java |    65 +
 .../apache/storm/generated/ProfileAction.java   |    74 +
 .../apache/storm/generated/ProfileRequest.java  |   631 +
 .../storm/generated/ReadableBlobMeta.java       |   510 +
 .../storm/generated/RebalanceOptions.java       |   664 +
 .../storm/generated/SettableBlobMeta.java       |   567 +
 .../apache/storm/generated/ShellComponent.java  |   516 +
 .../storm/generated/SpecificAggregateStats.java |   387 +
 .../storm/generated/SpoutAggregateStats.java    |   407 +
 .../org/apache/storm/generated/SpoutSpec.java   |   514 +
 .../org/apache/storm/generated/SpoutStats.java  |   917 +
 .../apache/storm/generated/StateSpoutSpec.java  |   514 +
 .../org/apache/storm/generated/StormBase.java   |  1382 +
 .../apache/storm/generated/StormTopology.java   |  1272 +
 .../org/apache/storm/generated/StreamInfo.java  |   554 +
 .../apache/storm/generated/SubmitOptions.java   |   533 +
 .../apache/storm/generated/SupervisorInfo.java  |  1446 +
 .../storm/generated/SupervisorPageInfo.java     |   624 +
 .../storm/generated/SupervisorSummary.java      |  1265 +
 .../storm/generated/ThriftSerializedObject.java |   516 +
 .../storm/generated/TopologyActionOptions.java  |   387 +
 .../storm/generated/TopologyHistoryInfo.java    |   461 +
 .../apache/storm/generated/TopologyInfo.java    |  2144 +
 .../storm/generated/TopologyInitialStatus.java  |    62 +
 .../storm/generated/TopologyPageInfo.java       |  2757 +
 .../apache/storm/generated/TopologyStats.java   |  1094 +
 .../apache/storm/generated/TopologyStatus.java  |    68 +
 .../apache/storm/generated/TopologySummary.java |  1901 +
 .../apache/storm/generated/WorkerResources.java |   605 +
 .../apache/storm/generated/WorkerSummary.java   |  1880 +
 .../storm/grouping/CustomStreamGrouping.java    |    43 +
 .../src/jvm/org/apache/storm/grouping/Load.java |    77 +
 .../grouping/LoadAwareCustomStreamGrouping.java |    24 +
 .../grouping/LoadAwareShuffleGrouping.java      |    76 +
 .../org/apache/storm/grouping/LoadMapping.java  |    64 +
 .../storm/grouping/PartialKeyGrouping.java      |   106 +
 .../apache/storm/grouping/ShuffleGrouping.java  |    63 +
 .../org/apache/storm/hooks/BaseTaskHook.java    |    61 +
 .../org/apache/storm/hooks/BaseWorkerHook.java  |    51 +
 .../jvm/org/apache/storm/hooks/ITaskHook.java   |    38 +
 .../jvm/org/apache/storm/hooks/IWorkerHook.java |    44 +
 .../storm/hooks/SubmitterHookException.java     |    41 +
 .../apache/storm/hooks/info/BoltAckInfo.java    |    40 +
 .../storm/hooks/info/BoltExecuteInfo.java       |    40 +
 .../apache/storm/hooks/info/BoltFailInfo.java   |    40 +
 .../org/apache/storm/hooks/info/EmitInfo.java   |    44 +
 .../apache/storm/hooks/info/SpoutAckInfo.java   |    39 +
 .../apache/storm/hooks/info/SpoutFailInfo.java  |    39 +
 .../storm/messaging/ConnectionWithStatus.java   |    49 +
 .../DeserializingConnectionCallback.java        |    60 +
 .../org/apache/storm/messaging/IConnection.java |    63 +
 .../storm/messaging/IConnectionCallback.java    |    31 +
 .../org/apache/storm/messaging/IContext.java    |    59 +
 .../org/apache/storm/messaging/TaskMessage.java |    53 +
 .../storm/messaging/TransportFactory.java       |    57 +
 .../apache/storm/messaging/local/Context.java   |   218 +
 .../apache/storm/messaging/netty/Client.java    |   612 +
 .../apache/storm/messaging/netty/Context.java   |   113 +
 .../storm/messaging/netty/ControlMessage.java   |    75 +
 .../messaging/netty/INettySerializable.java     |    26 +
 .../storm/messaging/netty/ISaslClient.java      |    28 +
 .../storm/messaging/netty/ISaslServer.java      |    26 +
 .../apache/storm/messaging/netty/IServer.java   |    26 +
 .../netty/KerberosSaslClientHandler.java        |   154 +
 .../netty/KerberosSaslNettyClient.java          |   202 +
 .../netty/KerberosSaslNettyClientState.java     |    31 +
 .../netty/KerberosSaslNettyServer.java          |   209 +
 .../netty/KerberosSaslNettyServerState.java     |    30 +
 .../netty/KerberosSaslServerHandler.java        |   133 +
 .../org/apache/storm/messaging/netty/Login.java |   411 +
 .../storm/messaging/netty/MessageBatch.java     |   118 +
 .../storm/messaging/netty/MessageBuffer.java    |    58 +
 .../storm/messaging/netty/MessageDecoder.java   |   144 +
 .../storm/messaging/netty/MessageEncoder.java   |    43 +
 .../netty/NettyRenameThreadFactory.java         |    56 +
 .../netty/NettyUncaughtExceptionHandler.java    |    35 +
 .../storm/messaging/netty/SaslMessageToken.java |   114 +
 .../storm/messaging/netty/SaslNettyClient.java  |   154 +
 .../messaging/netty/SaslNettyClientState.java   |    31 +
 .../storm/messaging/netty/SaslNettyServer.java  |   157 +
 .../messaging/netty/SaslNettyServerState.java   |    30 +
 .../messaging/netty/SaslStormClientHandler.java |   158 +
 .../netty/SaslStormServerAuthorizeHandler.java  |    83 +
 .../messaging/netty/SaslStormServerHandler.java |   153 +
 .../apache/storm/messaging/netty/SaslUtils.java |    68 +
 .../apache/storm/messaging/netty/Server.java    |   274 +
 .../messaging/netty/StormClientHandler.java     |    90 +
 .../netty/StormClientPipelineFactory.java       |    56 +
 .../messaging/netty/StormServerHandler.java     |    74 +
 .../netty/StormServerPipelineFactory.java       |    57 +
 .../apache/storm/metric/EventLoggerBolt.java    |    67 +
 .../storm/metric/FileBasedEventLogger.java      |   122 +
 .../org/apache/storm/metric/IEventLogger.java   |    70 +
 .../storm/metric/LoggingMetricsConsumer.java    |    76 +
 .../storm/metric/MetricsConsumerBolt.java       |   146 +
 .../jvm/org/apache/storm/metric/SystemBolt.java |   161 +
 .../storm/metric/api/AssignableMetric.java      |    34 +
 .../apache/storm/metric/api/CombinedMetric.java |    38 +
 .../apache/storm/metric/api/CountMetric.java    |    39 +
 .../org/apache/storm/metric/api/ICombiner.java  |    23 +
 .../org/apache/storm/metric/api/IMetric.java    |    30 +
 .../storm/metric/api/IMetricsConsumer.java      |    80 +
 .../org/apache/storm/metric/api/IReducer.java   |    24 +
 .../storm/metric/api/IStatefulObject.java       |    22 +
 .../apache/storm/metric/api/MeanReducer.java    |    54 +
 .../storm/metric/api/MultiCountMetric.java      |    45 +
 .../storm/metric/api/MultiReducedMetric.java    |    50 +
 .../apache/storm/metric/api/ReducedMetric.java  |    38 +
 .../apache/storm/metric/api/StateMetric.java    |    31 +
 .../metric/api/rpc/AssignableShellMetric.java   |    30 +
 .../metric/api/rpc/CombinedShellMetric.java     |    31 +
 .../storm/metric/api/rpc/CountShellMetric.java  |    37 +
 .../storm/metric/api/rpc/IShellMetric.java      |    31 +
 .../metric/api/rpc/ReducedShellMetric.java      |    32 +
 .../apache/storm/metric/cgroup/CGroupCpu.java   |    70 +
 .../storm/metric/cgroup/CGroupCpuGuarantee.java |    52 +
 .../storm/metric/cgroup/CGroupMemoryLimit.java  |    39 +
 .../storm/metric/cgroup/CGroupMemoryUsage.java  |    39 +
 .../storm/metric/cgroup/CGroupMetricsBase.java  |   114 +
 .../storm/metric/filter/FilterByMetricName.java |   110 +
 .../storm/metric/filter/MetricsFilter.java      |    26 +
 .../metric/internal/CountStatAndMetric.java     |   211 +
 .../metric/internal/LatencyStatAndMetric.java   |   262 +
 .../storm/metric/internal/MetricStatTimer.java  |    27 +
 .../internal/MultiCountStatAndMetric.java       |   112 +
 .../internal/MultiLatencyStatAndMetric.java     |   109 +
 .../storm/metric/internal/RateTracker.java      |   139 +
 .../storm/metric/util/DataPointExpander.java    |    79 +
 .../jvm/org/apache/storm/multilang/BoltMsg.java |    79 +
 .../org/apache/storm/multilang/ISerializer.java |    82 +
 .../apache/storm/multilang/JsonSerializer.java  |   203 +
 .../storm/multilang/NoOutputException.java      |    40 +
 .../org/apache/storm/multilang/ShellMsg.java    |   184 +
 .../org/apache/storm/multilang/SpoutMsg.java    |    50 +
 .../AbstractDNSToSwitchMapping.java             |    96 +
 .../networktopography/DNSToSwitchMapping.java   |    49 +
 .../DefaultRackDNSToSwitchMapping.java          |    52 +
 .../jvm/org/apache/storm/nimbus/NimbusInfo.java |   120 +
 .../apache/storm/pacemaker/PacemakerClient.java |   274 +
 .../storm/pacemaker/PacemakerClientHandler.java |    80 +
 .../storm/pacemaker/PacemakerClientPool.java    |   113 +
 .../pacemaker/PacemakerConnectionException.java |    24 +
 .../storm/pacemaker/codec/ThriftDecoder.java    |    69 +
 .../storm/pacemaker/codec/ThriftEncoder.java    |   109 +
 .../pacemaker/codec/ThriftNettyClientCodec.java |    97 +
 .../jvm/org/apache/storm/scheduler/Cluster.java |   857 +
 .../apache/storm/scheduler/ExecutorDetails.java |    54 +
 .../jvm/org/apache/storm/scheduler/INimbus.java |    49 +
 .../org/apache/storm/scheduler/IScheduler.java  |    40 +
 .../org/apache/storm/scheduler/ISupervisor.java |    45 +
 .../storm/scheduler/SchedulerAssignment.java    |    61 +
 .../scheduler/SchedulerAssignmentImpl.java      |   135 +
 .../storm/scheduler/SupervisorDetails.java      |   138 +
 .../org/apache/storm/scheduler/Topologies.java  |    87 +
 .../apache/storm/scheduler/TopologyDetails.java |   517 +
 .../org/apache/storm/scheduler/WorkerSlot.java  |    81 +
 .../storm/scheduler/resource/Component.java     |    54 +
 .../storm/scheduler/resource/RAS_Node.java      |   529 +
 .../storm/scheduler/resource/RAS_Nodes.java     |   138 +
 .../storm/scheduler/resource/ResourceUtils.java |   207 +
 .../scheduler/resource/SchedulingResult.java    |   116 +
 .../scheduler/resource/SchedulingState.java     |    56 +
 .../scheduler/resource/SchedulingStatus.java    |    40 +
 .../apache/storm/scheduler/resource/User.java   |   350 +
 .../DefaultResourceAwareStrategy.java           |   757 +
 .../strategies/scheduling/IStrategy.java        |    47 +
 .../storm/security/INimbusCredentialPlugin.java |    47 +
 .../auth/AbstractSaslClientCallbackHandler.java |    76 +
 .../auth/AbstractSaslServerCallbackHandler.java |    94 +
 .../apache/storm/security/auth/AuthUtils.java   |   419 +
 .../org/apache/storm/security/auth/AutoSSL.java |   161 +
 .../auth/DefaultHttpCredentialsPlugin.java      |    96 +
 .../security/auth/DefaultPrincipalToLocal.java  |    42 +
 .../apache/storm/security/auth/IAuthorizer.java |    53 +
 .../storm/security/auth/IAutoCredentials.java   |    55 +
 .../security/auth/ICredentialsRenewer.java      |    40 +
 .../auth/IGroupMappingServiceProvider.java      |    42 +
 .../security/auth/IHttpCredentialsPlugin.java   |    48 +
 .../storm/security/auth/IPrincipalToLocal.java  |    41 +
 .../storm/security/auth/ITransportPlugin.java   |    57 +
 .../security/auth/KerberosPrincipalToLocal.java |    45 +
 .../storm/security/auth/NimbusPrincipal.java    |    29 +
 .../apache/storm/security/auth/ReqContext.java  |   152 +
 .../security/auth/SaslTransportPlugin.java      |   180 +
 .../security/auth/ShellBasedGroupsMapping.java  |    99 +
 .../security/auth/SimpleTransportPlugin.java    |   163 +
 .../security/auth/SingleUserPrincipal.java      |    53 +
 .../storm/security/auth/TBackoffConnect.java    |    82 +
 .../storm/security/auth/ThriftClient.java       |   124 +
 .../security/auth/ThriftConnectionType.java     |    86 +
 .../storm/security/auth/ThriftServer.java       |    78 +
 .../auth/authorizer/DRPCAuthorizerBase.java     |    63 +
 .../authorizer/DRPCSimpleACLAuthorizer.java     |   177 +
 .../auth/authorizer/DenyAuthorizer.java         |    47 +
 .../authorizer/ImpersonationAuthorizer.java     |   172 +
 .../auth/authorizer/NoopAuthorizer.java         |    47 +
 .../auth/authorizer/SimpleACLAuthorizer.java    |   173 +
 .../authorizer/SimpleWhitelistAuthorizer.java   |    60 +
 .../auth/digest/ClientCallbackHandler.java      |    58 +
 .../auth/digest/DigestSaslTransportPlugin.java  |    68 +
 .../auth/digest/ServerCallbackHandler.java      |    85 +
 .../storm/security/auth/kerberos/AutoTGT.java   |   269 +
 .../auth/kerberos/AutoTGTKrb5LoginModule.java   |   112 +
 .../kerberos/AutoTGTKrb5LoginModuleTest.java    |    44 +
 .../auth/kerberos/ClientCallbackHandler.java    |   104 +
 .../kerberos/KerberosSaslTransportPlugin.java   |   262 +
 .../security/auth/kerberos/NoOpTTrasport.java   |    54 +
 .../auth/kerberos/ServerCallbackHandler.java    |    96 +
 .../auth/kerberos/jaas_kerberos_cluster.conf    |    49 +
 .../auth/kerberos/jaas_kerberos_launcher.conf   |    31 +
 .../auth/plain/PlainClientCallbackHandler.java  |    31 +
 .../auth/plain/PlainSaslTransportPlugin.java    |    71 +
 .../auth/plain/PlainServerCallbackHandler.java  |    55 +
 .../security/auth/plain/SaslPlainServer.java    |   158 +
 .../serialization/BlowfishTupleSerializer.java  |    92 +
 .../storm/serialization/DefaultKryoFactory.java |    64 +
 .../DefaultSerializationDelegate.java           |    58 +
 .../GzipBridgeSerializationDelegate.java        |    65 +
 .../GzipBridgeThriftSerializationDelegate.java  |    64 +
 .../GzipSerializationDelegate.java              |    64 +
 .../GzipThriftSerializationDelegate.java        |    57 +
 .../storm/serialization/IKryoDecorator.java     |    23 +
 .../storm/serialization/IKryoFactory.java       |    40 +
 .../storm/serialization/ITupleDeserializer.java |    24 +
 .../storm/serialization/ITupleSerializer.java   |    26 +
 .../serialization/KryoTupleDeserializer.java    |    56 +
 .../serialization/KryoTupleSerializer.java      |    60 +
 .../serialization/KryoValuesDeserializer.java   |    50 +
 .../serialization/KryoValuesSerializer.java     |    58 +
 .../serialization/SerializableSerializer.java   |    61 +
 .../serialization/SerializationDelegate.java    |    35 +
 .../serialization/SerializationFactory.java     |   241 +
 .../ThriftSerializationDelegate.java            |    52 +
 .../types/ArrayListSerializer.java              |    32 +
 .../serialization/types/HashMapSerializer.java  |    32 +
 .../serialization/types/HashSetSerializer.java  |    32 +
 .../types/ListDelegateSerializer.java           |    32 +
 .../org/apache/storm/spout/CheckPointState.java |   172 +
 .../org/apache/storm/spout/CheckpointSpout.java |   232 +
 .../storm/spout/IMultiSchemableSpout.java       |    23 +
 .../org/apache/storm/spout/ISchemableSpout.java |    24 +
 .../src/jvm/org/apache/storm/spout/ISpout.java  |   105 +
 .../storm/spout/ISpoutOutputCollector.java      |    32 +
 .../apache/storm/spout/ISpoutWaitStrategy.java  |    34 +
 .../jvm/org/apache/storm/spout/MultiScheme.java |    29 +
 .../storm/spout/NothingEmptyEmitStrategy.java   |    31 +
 .../org/apache/storm/spout/RawMultiScheme.java  |    38 +
 .../jvm/org/apache/storm/spout/RawScheme.java   |    36 +
 .../src/jvm/org/apache/storm/spout/Scheme.java  |    29 +
 .../apache/storm/spout/SchemeAsMultiScheme.java |    42 +
 .../jvm/org/apache/storm/spout/ShellSpout.java  |   327 +
 .../storm/spout/SleepSpoutWaitStrategy.java     |    41 +
 .../storm/spout/SpoutOutputCollector.java       |   144 +
 .../storm/state/DefaultStateSerializer.java     |    77 +
 .../jvm/org/apache/storm/state/IStateSpout.java |    29 +
 .../storm/state/IStateSpoutOutputCollector.java |    22 +
 .../apache/storm/state/ISubscribedState.java    |    25 +
 .../state/ISynchronizeOutputCollector.java      |    24 +
 .../storm/state/InMemoryKeyValueState.java      |   126 +
 .../state/InMemoryKeyValueStateProvider.java    |    43 +
 .../org/apache/storm/state/KeyValueState.java   |    58 +
 .../jvm/org/apache/storm/state/Serializer.java  |    30 +
 .../src/jvm/org/apache/storm/state/State.java   |    55 +
 .../org/apache/storm/state/StateFactory.java    |    69 +
 .../org/apache/storm/state/StateProvider.java   |    38 +
 .../storm/state/StateSpoutOutputCollector.java  |    28 +
 .../storm/state/SynchronizeOutputCollector.java |    30 +
 .../apache/storm/stats/BoltExecutorStats.java   |   106 +
 .../jvm/org/apache/storm/stats/CommonStats.java |   114 +
 .../apache/storm/stats/SpoutExecutorStats.java  |    76 +
 .../jvm/org/apache/storm/stats/StatsUtil.java   |  2610 +
 .../src/jvm/org/apache/storm/streams/Edge.java  |    41 +
 .../org/apache/storm/streams/GroupingInfo.java  |   100 +
 .../src/jvm/org/apache/storm/streams/Node.java  |   170 +
 .../src/jvm/org/apache/storm/streams/Pair.java  |   103 +
 .../org/apache/storm/streams/PairStream.java    |   525 +
 .../org/apache/storm/streams/PartitionNode.java |    37 +
 .../org/apache/storm/streams/ProcessorBolt.java |    70 +
 .../storm/streams/ProcessorBoltDelegate.java    |   348 +
 .../org/apache/storm/streams/ProcessorNode.java |    83 +
 .../apache/storm/streams/RefCountedTuple.java   |    64 +
 .../jvm/org/apache/storm/streams/SinkNode.java  |    44 +
 .../jvm/org/apache/storm/streams/SpoutNode.java |    47 +
 .../storm/streams/StatefulProcessorBolt.java    |   116 +
 .../jvm/org/apache/storm/streams/Stream.java    |   515 +
 .../org/apache/storm/streams/StreamBolt.java    |    38 +
 .../org/apache/storm/streams/StreamBuilder.java |   602 +
 .../org/apache/storm/streams/StreamState.java   |    43 +
 .../org/apache/storm/streams/StreamUtil.java    |    69 +
 .../storm/streams/StreamsEdgeFactory.java       |    29 +
 .../org/apache/storm/streams/UniqueIdGen.java   |    56 +
 .../org/apache/storm/streams/WindowNode.java    |    39 +
 .../storm/streams/WindowedProcessorBolt.java    |   137 +
 .../storm/streams/operations/BiFunction.java    |    37 +
 .../streams/operations/CombinerAggregator.java  |    97 +
 .../storm/streams/operations/Consumer.java      |    32 +
 .../streams/operations/FlatMapFunction.java     |    27 +
 .../storm/streams/operations/Function.java      |    34 +
 .../streams/operations/IdentityFunction.java    |    31 +
 .../storm/streams/operations/Operation.java     |    26 +
 .../streams/operations/PairFlatMapFunction.java |    30 +
 .../storm/streams/operations/PairFunction.java  |    30 +
 .../streams/operations/PairValueJoiner.java     |    40 +
 .../storm/streams/operations/Predicate.java     |    33 +
 .../storm/streams/operations/PrintConsumer.java |    30 +
 .../storm/streams/operations/Reducer.java       |    35 +
 .../storm/streams/operations/StateUpdater.java  |    67 +
 .../storm/streams/operations/ValueJoiner.java   |    36 +
 .../streams/operations/aggregators/Count.java   |    47 +
 .../streams/operations/aggregators/LongSum.java |    45 +
 .../operations/mappers/PairValueMapper.java     |    51 +
 .../operations/mappers/TupleValueMapper.java    |    30 +
 .../operations/mappers/TupleValueMappers.java   |   174 +
 .../streams/operations/mappers/ValueMapper.java |    45 +
 .../operations/mappers/ValuesMapper.java        |    48 +
 .../processors/AggregateByKeyProcessor.java     |    78 +
 .../streams/processors/AggregateProcessor.java  |    69 +
 .../storm/streams/processors/BaseProcessor.java |   109 +
 .../streams/processors/BatchProcessor.java      |    25 +
 .../streams/processors/BranchProcessor.java     |    41 +
 .../processors/ChainedProcessorContext.java     |    66 +
 .../processors/EmittingProcessorContext.java    |   155 +
 .../streams/processors/FilterProcessor.java     |    35 +
 .../streams/processors/FlatMapProcessor.java    |    35 +
 .../processors/FlatMapValuesProcessor.java      |    36 +
 .../streams/processors/ForEachProcessor.java    |    33 +
 .../processors/ForwardingProcessorContext.java  |   102 +
 .../storm/streams/processors/JoinProcessor.java |   152 +
 .../storm/streams/processors/MapProcessor.java  |    33 +
 .../streams/processors/MapValuesProcessor.java  |    34 +
 .../MergeAggregateByKeyProcessor.java           |    54 +
 .../processors/MergeAggregateProcessor.java     |    47 +
 .../storm/streams/processors/PeekProcessor.java |    34 +
 .../storm/streams/processors/Processor.java     |    54 +
 .../streams/processors/ProcessorContext.java    |    62 +
 .../processors/ReduceByKeyProcessor.java        |    52 +
 .../streams/processors/ReduceProcessor.java     |    43 +
 .../streams/processors/StateQueryProcessor.java |    48 +
 .../streams/processors/StatefulProcessor.java   |    36 +
 .../processors/UpdateStateByKeyProcessor.java   |    49 +
 .../org/apache/storm/streams/tuple/Tuple10.java |   112 +
 .../org/apache/storm/streams/tuple/Tuple3.java  |    70 +
 .../org/apache/storm/streams/tuple/Tuple4.java  |    76 +
 .../org/apache/storm/streams/tuple/Tuple5.java  |    82 +
 .../org/apache/storm/streams/tuple/Tuple6.java  |    89 +
 .../org/apache/storm/streams/tuple/Tuple7.java  |    94 +
 .../org/apache/storm/streams/tuple/Tuple8.java  |   100 +
 .../org/apache/storm/streams/tuple/Tuple9.java  |   106 +
 .../storm/streams/windowing/BaseWindow.java     |    64 +
 .../storm/streams/windowing/SlidingWindows.java |   151 +
 .../streams/windowing/TumblingWindows.java      |   119 +
 .../apache/storm/streams/windowing/Window.java  |    70 +
 .../storm/task/GeneralTopologyContext.java      |   202 +
 .../src/jvm/org/apache/storm/task/IBolt.java    |    84 +
 .../org/apache/storm/task/IErrorReporter.java   |    22 +
 .../org/apache/storm/task/IMetricsContext.java  |    31 +
 .../org/apache/storm/task/IOutputCollector.java |    33 +
 .../org/apache/storm/task/OutputCollector.java  |   236 +
 .../jvm/org/apache/storm/task/ShellBolt.java    |   435 +
 .../org/apache/storm/task/TopologyContext.java  |   401 +
 .../storm/task/WorkerTopologyContext.java       |   106 +
 .../apache/storm/testing/AckFailDelegate.java   |    25 +
 .../apache/storm/testing/AckFailMapTracker.java |    52 +
 .../org/apache/storm/testing/AckTracker.java    |    52 +
 .../AlternateRackDNSToSwitchMapping.java        |    65 +
 .../apache/storm/testing/BatchNumberList.java   |    72 +
 .../apache/storm/testing/BatchProcessWord.java  |    39 +
 .../org/apache/storm/testing/BatchRepeatA.java  |    48 +
 .../org/apache/storm/testing/BoltTracker.java   |    42 +
 .../apache/storm/testing/CompletableSpout.java  |    39 +
 .../apache/storm/testing/CountingBatchBolt.java |    55 +
 .../storm/testing/CountingCommitBolt.java       |    57 +
 .../org/apache/storm/testing/FeederSpout.java   |   107 +
 .../org/apache/storm/testing/FixedTuple.java    |    43 +
 .../apache/storm/testing/FixedTupleSpout.java   |   185 +
 .../testing/ForwardingMetricsConsumer.java      |   100 +
 .../org/apache/storm/testing/IdentityBolt.java  |    42 +
 .../apache/storm/testing/IntegrationTest.java   |    38 +
 .../storm/testing/KeyedCountingBatchBolt.java   |    62 +
 .../testing/KeyedCountingCommitterBolt.java     |    24 +
 .../storm/testing/KeyedSummingBatchBolt.java    |    61 +
 .../storm/testing/MemoryTransactionalSpout.java |   182 +
 .../testing/MemoryTransactionalSpoutMeta.java   |    38 +
 .../apache/storm/testing/MkClusterParam.java    |    69 +
 .../org/apache/storm/testing/MkTupleParam.java  |    58 +
 .../org/apache/storm/testing/MockedSources.java |    68 +
 .../jvm/org/apache/storm/testing/NGrouping.java |    48 +
 .../storm/testing/NonRichBoltTracker.java       |    51 +
 .../testing/OpaqueMemoryTransactionalSpout.java |   188 +
 .../apache/storm/testing/PrepareBatchBolt.java  |    53 +
 .../storm/testing/PythonShellMetricsBolt.java   |    53 +
 .../storm/testing/PythonShellMetricsSpout.java  |    56 +
 .../testing/SingleUserSimpleTransport.java      |    37 +
 .../org/apache/storm/testing/SpoutTracker.java  |   111 +
 .../storm/testing/TestAggregatesCounter.java    |    63 +
 .../org/apache/storm/testing/TestConfBolt.java  |    62 +
 .../apache/storm/testing/TestEventLogSpout.java |   143 +
 .../storm/testing/TestEventOrderCheckBolt.java  |    76 +
 .../apache/storm/testing/TestGlobalCount.java   |    60 +
 .../apache/storm/testing/TestKryoDecorator.java |    31 +
 .../apache/storm/testing/TestPlannerBolt.java   |    45 +
 .../apache/storm/testing/TestPlannerSpout.java  |    86 +
 .../org/apache/storm/testing/TestSerObject.java |    37 +
 .../storm/testing/TestWordBytesCounter.java     |    27 +
 .../apache/storm/testing/TestWordCounter.java   |    65 +
 .../org/apache/storm/testing/TestWordSpout.java |    87 +
 .../jvm/org/apache/storm/testing/TmpPath.java   |    69 +
 .../apache/storm/testing/TupleCaptureBolt.java  |    83 +
 .../topology/BaseConfigurationDeclarer.java     |    83 +
 .../topology/BaseStatefulBoltExecutor.java      |   209 +
 .../storm/topology/BasicBoltExecutor.java       |    67 +
 .../storm/topology/BasicOutputCollector.java    |    72 +
 .../org/apache/storm/topology/BoltDeclarer.java |    26 +
 .../topology/CheckpointTupleForwarder.java      |   103 +
 .../ComponentConfigurationDeclarer.java         |    29 +
 .../apache/storm/topology/FailedException.java  |    36 +
 .../org/apache/storm/topology/IBasicBolt.java   |    33 +
 .../storm/topology/IBasicOutputCollector.java   |    29 +
 .../org/apache/storm/topology/IComponent.java   |    44 +
 .../org/apache/storm/topology/IRichBolt.java    |    29 +
 .../org/apache/storm/topology/IRichSpout.java   |    29 +
 .../apache/storm/topology/IRichStateSpout.java  |    25 +
 .../apache/storm/topology/IStatefulBolt.java    |    48 +
 .../storm/topology/IStatefulComponent.java      |    56 +
 .../storm/topology/IStatefulWindowedBolt.java   |    26 +
 .../apache/storm/topology/IWindowedBolt.java    |    51 +
 .../apache/storm/topology/InputDeclarer.java    |   184 +
 .../storm/topology/OutputFieldsDeclarer.java    |    32 +
 .../storm/topology/OutputFieldsGetter.java      |    53 +
 .../storm/topology/ReportedFailedException.java |    36 +
 .../apache/storm/topology/ResourceDeclarer.java |    28 +
 .../apache/storm/topology/SpoutDeclarer.java    |    22 +
 .../storm/topology/StatefulBoltExecutor.java    |   193 +
 .../topology/StatefulWindowedBoltExecutor.java  |   372 +
 .../apache/storm/topology/TopologyBuilder.java  |   587 +
 .../topology/TupleFieldTimestampExtractor.java  |    48 +
 .../storm/topology/WindowedBoltExecutor.java    |   363 +
 .../storm/topology/base/BaseBasicBolt.java      |    33 +
 .../storm/topology/base/BaseBatchBolt.java      |    24 +
 .../storm/topology/base/BaseComponent.java      |    28 +
 ...BaseOpaquePartitionedTransactionalSpout.java |    25 +
 .../base/BasePartitionedTransactionalSpout.java |    24 +
 .../storm/topology/base/BaseRichBolt.java       |    26 +
 .../storm/topology/base/BaseRichSpout.java      |    46 +
 .../storm/topology/base/BaseStatefulBolt.java   |    64 +
 .../topology/base/BaseStatefulWindowedBolt.java |   168 +
 .../base/BaseTickTupleAwareRichBolt.java        |    60 +
 .../topology/base/BaseTransactionalBolt.java    |    24 +
 .../topology/base/BaseTransactionalSpout.java   |    24 +
 .../storm/topology/base/BaseWindowedBolt.java   |   365 +
 .../apache/storm/transactional/ICommitter.java  |    26 +
 .../ICommitterTransactionalSpout.java           |    31 +
 .../transactional/ITransactionalSpout.java      |    94 +
 .../storm/transactional/TransactionAttempt.java |    61 +
 .../TransactionalSpoutBatchExecutor.java        |    96 +
 .../TransactionalSpoutCoordinator.java          |   214 +
 .../TransactionalTopologyBuilder.java           |   521 +
 .../IOpaquePartitionedTransactionalSpout.java   |    56 +
 .../IPartitionedTransactionalSpout.java         |    69 +
 ...uePartitionedTransactionalSpoutExecutor.java |   160 +
 .../PartitionedTransactionalSpoutExecutor.java  |   142 +
 .../state/RotatingTransactionalState.java       |   149 +
 .../state/TestTransactionalState.java           |    47 +
 .../transactional/state/TransactionalState.java |   172 +
 .../apache/storm/trident/JoinOutFieldsMode.java |    33 +
 .../jvm/org/apache/storm/trident/JoinType.java  |    30 +
 .../jvm/org/apache/storm/trident/Stream.java    |   897 +
 .../org/apache/storm/trident/TridentState.java  |    59 +
 .../apache/storm/trident/TridentTopology.java   |   986 +
 .../trident/drpc/ReturnResultsReducer.java      |   129 +
 .../fluent/ChainedAggregatorDeclarer.java       |   183 +
 .../fluent/ChainedFullAggregatorDeclarer.java   |    32 +
 .../ChainedPartitionAggregatorDeclarer.java     |    32 +
 .../trident/fluent/GlobalAggregationScheme.java |    26 +
 .../storm/trident/fluent/GroupedStream.java     |   174 +
 .../trident/fluent/IAggregatableStream.java     |    31 +
 .../fluent/IChainedAggregatorDeclarer.java      |    24 +
 .../storm/trident/fluent/UniqueIdGen.java       |    38 +
 .../storm/trident/graph/GraphGrouper.java       |   119 +
 .../org/apache/storm/trident/graph/Group.java   |   150 +
 .../storm/trident/operation/Aggregator.java     |    26 +
 .../storm/trident/operation/Assembly.java       |    45 +
 .../storm/trident/operation/BaseAggregator.java |    23 +
 .../storm/trident/operation/BaseFilter.java     |    23 +
 .../storm/trident/operation/BaseFunction.java   |    23 +
 .../trident/operation/BaseMultiReducer.java     |    33 +
 .../storm/trident/operation/BaseOperation.java  |    47 +
 .../trident/operation/CombinerAggregator.java   |    29 +
 .../storm/trident/operation/Consumer.java       |    35 +
 .../operation/DefaultResourceDeclarer.java      |    68 +
 .../storm/trident/operation/EachOperation.java  |    22 +
 .../apache/storm/trident/operation/Filter.java  |    48 +
 .../trident/operation/FlatMapFunction.java      |    37 +
 .../storm/trident/operation/Function.java       |    92 +
 .../trident/operation/GroupedMultiReducer.java  |    31 +
 .../trident/operation/ITridentResource.java     |    32 +
 .../storm/trident/operation/MapFunction.java    |    36 +
 .../storm/trident/operation/MultiReducer.java   |    31 +
 .../storm/trident/operation/Operation.java      |    46 +
 .../OperationAwareFlatMapFunction.java          |    24 +
 .../operation/OperationAwareMapFunction.java    |    24 +
 .../trident/operation/ReducerAggregator.java    |    26 +
 .../trident/operation/TridentCollector.java     |    52 +
 .../operation/TridentMultiReducerContext.java   |    36 +
 .../operation/TridentOperationContext.java      |    65 +
 .../operation/builtin/ComparisonAggregator.java |    91 +
 .../storm/trident/operation/builtin/Count.java  |    41 +
 .../storm/trident/operation/builtin/Debug.java  |    62 +
 .../storm/trident/operation/builtin/Equals.java |    38 +
 .../trident/operation/builtin/FilterNull.java   |    35 +
 .../storm/trident/operation/builtin/FirstN.java |   130 +
 .../storm/trident/operation/builtin/MapGet.java |    38 +
 .../storm/trident/operation/builtin/Max.java    |    37 +
 .../operation/builtin/MaxWithComparator.java    |    51 +
 .../storm/trident/operation/builtin/Min.java    |    36 +
 .../operation/builtin/MinWithComparator.java    |    51 +
 .../storm/trident/operation/builtin/Negate.java |    76 +
 .../trident/operation/builtin/SnapshotGet.java  |    44 +
 .../storm/trident/operation/builtin/Sum.java    |    42 +
 .../operation/builtin/TupleCollectionGet.java   |    46 +
 .../operation/impl/CaptureCollector.java        |    42 +
 .../operation/impl/ChainedAggregatorImpl.java   |   113 +
 .../trident/operation/impl/ChainedResult.java   |    53 +
 .../operation/impl/CombinerAggStateUpdater.java |    56 +
 .../impl/CombinerAggregatorCombineImpl.java     |    61 +
 .../impl/CombinerAggregatorInitImpl.java        |    49 +
 .../operation/impl/ConsumerExecutor.java        |    38 +
 .../trident/operation/impl/FilterExecutor.java  |    53 +
 .../operation/impl/FlatMapFunctionExecutor.java |    64 +
 .../operation/impl/GlobalBatchToPartition.java  |    29 +
 .../trident/operation/impl/GroupCollector.java  |    48 +
 .../operation/impl/GroupedAggregator.java       |    96 +
 .../impl/GroupedMultiReducerExecutor.java       |    95 +
 .../operation/impl/IdentityMultiReducer.java    |    51 +
 .../impl/IndexHashBatchToPartition.java         |    29 +
 .../storm/trident/operation/impl/JoinState.java |    39 +
 .../operation/impl/JoinerMultiReducer.java      |   142 +
 .../operation/impl/MapFunctionExecutor.java     |    62 +
 ...PreservingFieldsOrderJoinerMultiReducer.java |   175 +
 .../operation/impl/ReducerAggStateUpdater.java  |    53 +
 .../operation/impl/ReducerAggregatorImpl.java   |    56 +
 .../storm/trident/operation/impl/Result.java    |    27 +
 .../operation/impl/SingleEmitAggregator.java    |    95 +
 .../trident/operation/impl/TrueFilter.java      |    40 +
 .../storm/trident/partition/GlobalGrouping.java |    42 +
 .../trident/partition/IdentityGrouping.java     |    59 +
 .../trident/partition/IndexHashGrouping.java    |    55 +
 .../storm/trident/planner/BridgeReceiver.java   |    38 +
 .../org/apache/storm/trident/planner/Node.java  |    69 +
 .../storm/trident/planner/NodeStateInfo.java    |    31 +
 .../storm/trident/planner/PartitionNode.java    |    50 +
 .../storm/trident/planner/ProcessorContext.java |    29 +
 .../storm/trident/planner/ProcessorNode.java    |    38 +
 .../apache/storm/trident/planner/SpoutNode.java |    39 +
 .../storm/trident/planner/SubtopologyBolt.java  |   246 +
 .../storm/trident/planner/TridentProcessor.java |    40 +
 .../storm/trident/planner/TupleReceiver.java    |    27 +
 .../planner/processor/AggregateProcessor.java   |    84 +
 .../planner/processor/AppendCollector.java      |    62 +
 .../planner/processor/EachProcessor.java        |    80 +
 .../planner/processor/FreshCollector.java       |    59 +
 .../trident/planner/processor/MapProcessor.java |    87 +
 .../processor/MultiReducerProcessor.java        |    93 +
 .../processor/PartitionPersistProcessor.java    |   107 +
 .../planner/processor/ProjectedProcessor.java   |    73 +
 .../planner/processor/StateQueryProcessor.java  |   107 +
 .../planner/processor/TridentContext.java       |    76 +
 .../storm/trident/spout/BatchSpoutExecutor.java |    92 +
 .../apache/storm/trident/spout/IBatchID.java    |    24 +
 .../apache/storm/trident/spout/IBatchSpout.java |    33 +
 .../trident/spout/ICommitterTridentSpout.java   |    31 +
 .../spout/IOpaquePartitionedTridentSpout.java   |    87 +
 .../trident/spout/IPartitionedTridentSpout.java |    77 +
 .../storm/trident/spout/ISpoutPartition.java    |    25 +
 .../storm/trident/spout/ITridentDataSource.java |    26 +
 .../storm/trident/spout/ITridentSpout.java      |   125 +
 .../OpaquePartitionedTridentSpoutExecutor.java  |   234 +
 .../spout/PartitionedTridentSpoutExecutor.java  |   188 +
 .../trident/spout/RichSpoutBatchExecutor.java   |   204 +
 .../storm/trident/spout/RichSpoutBatchId.java   |    49 +
 .../spout/RichSpoutBatchIdSerializer.java       |    38 +
 .../trident/spout/RichSpoutBatchTriggerer.java  |   183 +
 .../trident/spout/TridentSpoutCoordinator.java  |    94 +
 .../trident/spout/TridentSpoutExecutor.java     |   138 +
 .../storm/trident/state/BaseQueryFunction.java  |    25 +
 .../storm/trident/state/BaseStateUpdater.java   |    25 +
 .../trident/state/CombinerValueUpdater.java     |    45 +
 .../storm/trident/state/ITupleCollection.java   |    26 +
 .../state/JSONNonTransactionalSerializer.java   |    45 +
 .../trident/state/JSONOpaqueSerializer.java     |    52 +
 .../state/JSONTransactionalSerializer.java      |    50 +
 .../apache/storm/trident/state/OpaqueValue.java |    75 +
 .../storm/trident/state/QueryFunction.java      |    28 +
 .../storm/trident/state/ReadOnlyState.java      |    31 +
 .../trident/state/ReducerValueUpdater.java      |    41 +
 .../apache/storm/trident/state/Serializer.java  |    26 +
 .../org/apache/storm/trident/state/State.java   |    39 +
 .../storm/trident/state/StateFactory.java       |    26 +
 .../apache/storm/trident/state/StateSpec.java   |    30 +
 .../apache/storm/trident/state/StateType.java   |    25 +
 .../storm/trident/state/StateUpdater.java       |    33 +
 .../storm/trident/state/TransactionalValue.java |    44 +
 .../storm/trident/state/ValueUpdater.java       |    23 +
 .../trident/state/map/CachedBatchReadsMap.java  |    80 +
 .../storm/trident/state/map/CachedMap.java      |    78 +
 .../storm/trident/state/map/IBackingMap.java    |    26 +
 .../state/map/MapCombinerAggStateUpdater.java   |    84 +
 .../state/map/MapReducerAggStateUpdater.java    |    91 +
 .../storm/trident/state/map/MapState.java       |    26 +
 .../state/map/MicroBatchIBackingMap.java        |    85 +
 .../trident/state/map/NonTransactionalMap.java  |    67 +
 .../storm/trident/state/map/OpaqueMap.java      |   124 +
 .../trident/state/map/ReadOnlyMapState.java     |    26 +
 .../trident/state/map/RemovableMapState.java    |    25 +
 .../trident/state/map/SnapshottableMap.java     |    76 +
 .../trident/state/map/TransactionalMap.java     |   109 +
 .../state/snapshot/ReadOnlySnapshottable.java   |    24 +
 .../trident/state/snapshot/Snapshottable.java   |    27 +
 .../trident/testing/CountAsAggregator.java      |    47 +
 .../storm/trident/testing/FeederBatchSpout.java |   185 +
 .../testing/FeederCommitterBatchSpout.java      |    96 +
 .../storm/trident/testing/FixedBatchSpout.java  |    97 +
 .../apache/storm/trident/testing/IFeeder.java   |    23 +
 .../trident/testing/LRUMemoryMapState.java      |   154 +
 .../storm/trident/testing/MemoryBackingMap.java |    47 +
 .../storm/trident/testing/MemoryMapState.java   |   176 +
 .../org/apache/storm/trident/testing/Split.java |    36 +
 .../storm/trident/testing/StringLength.java     |    32 +
 .../storm/trident/testing/TrueFilter.java       |    30 +
 .../storm/trident/testing/TuplifyArgs.java      |    41 +
 .../storm/trident/topology/BatchInfo.java       |    33 +
 .../trident/topology/ITridentBatchBolt.java     |    32 +
 .../topology/MasterBatchCoordinator.java        |   316 +
 .../trident/topology/TransactionAttempt.java    |    66 +
 .../trident/topology/TridentBoltExecutor.java   |   440 +
 .../topology/TridentTopologyBuilder.java        |   751 +
 .../state/RotatingTransactionalState.java       |   186 +
 .../topology/state/TestTransactionalState.java  |    47 +
 .../topology/state/TransactionalState.java      |   192 +
 .../apache/storm/trident/tuple/ComboList.java   |    92 +
 .../apache/storm/trident/tuple/ConsList.java    |    60 +
 .../storm/trident/tuple/TridentTuple.java       |    34 +
 .../storm/trident/tuple/TridentTupleView.java   |   361 +
 .../storm/trident/tuple/ValuePointer.java       |    60 +
 .../storm/trident/util/ErrorEdgeFactory.java    |    28 +
 .../apache/storm/trident/util/IndexedEdge.java  |    50 +
 .../org/apache/storm/trident/util/LRUMap.java   |    35 +
 .../apache/storm/trident/util/TridentUtils.java |   115 +
 .../windowing/AbstractTridentWindowManager.java |   238 +
 .../windowing/ITridentWindowManager.java        |    59 +
 .../windowing/InMemoryTridentWindowManager.java |    72 +
 .../trident/windowing/InMemoryWindowsStore.java |   200 +
 .../windowing/InMemoryWindowsStoreFactory.java  |    44 +
 .../StoreBasedTridentWindowManager.java         |   217 +
 .../trident/windowing/TridentBatchTuple.java    |    42 +
 .../trident/windowing/WindowKryoSerializer.java |    87 +
 .../windowing/WindowTridentProcessor.java       |   265 +
 .../storm/trident/windowing/WindowsState.java   |    52 +
 .../trident/windowing/WindowsStateFactory.java  |    40 +
 .../trident/windowing/WindowsStateUpdater.java  |    81 +
 .../storm/trident/windowing/WindowsStore.java   |    79 +
 .../trident/windowing/WindowsStoreFactory.java  |    40 +
 .../windowing/config/BaseWindowConfig.java      |    48 +
 .../windowing/config/SlidingCountWindow.java    |    43 +
 .../windowing/config/SlidingDurationWindow.java |    44 +
 .../windowing/config/TumblingCountWindow.java   |    43 +
 .../config/TumblingDurationWindow.java          |    42 +
 .../trident/windowing/config/WindowConfig.java  |    57 +
 .../windowing/strategy/BaseWindowStrategy.java  |    32 +
 .../strategy/SlidingCountWindowStrategy.java    |    59 +
 .../strategy/SlidingDurationWindowStrategy.java |    60 +
 .../strategy/TumblingCountWindowStrategy.java   |    60 +
 .../TumblingDurationWindowStrategy.java         |    60 +
 .../windowing/strategy/WindowStrategy.java      |    45 +
 .../org/apache/storm/tuple/AddressedTuple.java  |    48 +
 .../src/jvm/org/apache/storm/tuple/Fields.java  |   142 +
 .../src/jvm/org/apache/storm/tuple/ITuple.java  |   214 +
 .../jvm/org/apache/storm/tuple/MessageId.java   |    93 +
 .../src/jvm/org/apache/storm/tuple/Tuple.java   |    68 +
 .../jvm/org/apache/storm/tuple/TupleImpl.java   |   356 +
 .../src/jvm/org/apache/storm/tuple/Values.java  |    37 +
 .../jvm/org/apache/storm/utils/BatchHelper.java |    92 +
 .../storm/utils/BufferFileInputStream.java      |    54 +
 .../apache/storm/utils/CRC32OutputStream.java   |    44 +
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   365 +
 .../org/apache/storm/utils/CuratorUtils.java    |   115 +
 .../jvm/org/apache/storm/utils/DRPCClient.java  |    66 +
 .../utils/DisruptorBackpressureCallback.java    |    27 +
 .../org/apache/storm/utils/DisruptorQueue.java  |   589 +
 .../storm/utils/ExtendedThreadPoolExecutor.java |    67 +
 .../jvm/org/apache/storm/utils/IPredicate.java  |    27 +
 .../storm/utils/IndifferentAccessMap.java       |   177 +
 .../org/apache/storm/utils/InprocMessaging.java |   100 +
 .../storm/utils/KeyedRoundRobinQueue.java       |    68 +
 .../org/apache/storm/utils/ListDelegate.java    |   156 +
 .../jvm/org/apache/storm/utils/LocalState.java  |   277 +
 .../jvm/org/apache/storm/utils/MutableInt.java  |    43 +
 .../jvm/org/apache/storm/utils/MutableLong.java |    43 +
 .../org/apache/storm/utils/MutableObject.java   |    38 +
 .../org/apache/storm/utils/NimbusClient.java    |   139 +
 .../utils/NimbusLeaderNotFoundException.java    |    41 +
 .../org/apache/storm/utils/ObjectReader.java    |   119 +
 .../org/apache/storm/utils/ReflectionUtils.java |    83 +
 .../storm/utils/RegisteredGlobalState.java      |    62 +
 .../jvm/org/apache/storm/utils/RotatingMap.java |   128 +
 .../org/apache/storm/utils/ServiceRegistry.java |    47 +
 .../storm/utils/ShellBoltMessageQueue.java      |   121 +
 .../org/apache/storm/utils/ShellProcess.java    |   215 +
 .../jvm/org/apache/storm/utils/ShellUtils.java  |   505 +
 .../StormBoundedExponentialBackoffRetry.java    |    79 +
 .../apache/storm/utils/ThriftTopologyUtils.java |    70 +
 .../src/jvm/org/apache/storm/utils/Time.java    |   235 +
 .../org/apache/storm/utils/TimeCacheMap.java    |   125 +
 .../org/apache/storm/utils/TransferDrainer.java |   140 +
 .../jvm/org/apache/storm/utils/TupleUtils.java  |    67 +
 .../src/jvm/org/apache/storm/utils/Utils.java   |  1273 +
 .../jvm/org/apache/storm/utils/VersionInfo.java |   131 +
 .../org/apache/storm/utils/VersionedStore.java  |   188 +
 .../storm/utils/WindowedTimeThrottler.java      |    51 +
 .../storm/utils/WorkerBackpressureCallback.java |    26 +
 .../storm/utils/WorkerBackpressureThread.java   |    80 +
 .../org/apache/storm/utils/WritableUtils.java   |   375 +
 .../apache/storm/utils/ZookeeperAuthInfo.java   |    53 +
 .../storm/utils/ZookeeperServerCnxnFactory.java |    84 +
 .../storm/validation/ConfigValidation.java      |   716 +
 .../validation/ConfigValidationAnnotations.java |   218 +
 .../storm/validation/ConfigValidationUtils.java |   175 +
 .../storm/windowing/CountEvictionPolicy.java    |    81 +
 .../storm/windowing/CountTriggerPolicy.java     |    76 +
 .../storm/windowing/DefaultEvictionContext.java |    53 +
 .../jvm/org/apache/storm/windowing/Event.java   |    49 +
 .../org/apache/storm/windowing/EventImpl.java   |    51 +
 .../apache/storm/windowing/EvictionContext.java |    47 +
 .../apache/storm/windowing/EvictionPolicy.java  |    82 +
 .../storm/windowing/TimeEvictionPolicy.java     |    72 +
 .../storm/windowing/TimeTriggerPolicy.java      |   132 +
 .../storm/windowing/TimestampExtractor.java     |    35 +
 .../apache/storm/windowing/TriggerHandler.java  |    31 +
 .../apache/storm/windowing/TriggerPolicy.java   |    49 +
 .../org/apache/storm/windowing/TupleWindow.java |    26 +
 .../apache/storm/windowing/TupleWindowImpl.java |    93 +
 .../apache/storm/windowing/WaterMarkEvent.java  |    38 +
 .../windowing/WaterMarkEventGenerator.java      |   121 +
 .../windowing/WatermarkCountEvictionPolicy.java |    68 +
 .../windowing/WatermarkCountTriggerPolicy.java  |    91 +
 .../windowing/WatermarkTimeEvictionPolicy.java  |    77 +
 .../windowing/WatermarkTimeTriggerPolicy.java   |   126 +
 .../jvm/org/apache/storm/windowing/Window.java  |    55 +
 .../windowing/WindowLifecycleListener.java      |    43 +
 .../apache/storm/windowing/WindowManager.java   |   289 +
 .../apache/storm/zookeeper/ClientZookeeper.java |   345 +
 .../apache/storm/zookeeper/ZkEventTypes.java    |    43 +
 .../apache/storm/zookeeper/ZkKeeperStates.java  |    41 +
 storm-client/src/py/__init__.py                 |    18 +
 storm-client/src/py/storm/DistributedRPC-remote |   112 +
 storm-client/src/py/storm/DistributedRPC.py     |   311 +
 .../py/storm/DistributedRPCInvocations-remote   |   133 +
 .../src/py/storm/DistributedRPCInvocations.py   |   886 +
 storm-client/src/py/storm/Nimbus-remote         |   420 +
 storm-client/src/py/storm/Nimbus.py             |  9915 ++++
 storm-client/src/py/storm/__init__.py           |    19 +
 storm-client/src/py/storm/constants.py          |    29 +
 storm-client/src/py/storm/ttypes.py             | 11695 ++++
 storm-client/src/py_license_header.txt          |    18 +
 .../storm-client-version-info.properties        |    24 +
 storm-client/src/storm.thrift                   |   775 +
 .../storm/PaceMakerStateStorageFactoryTest.java |   153 +
 .../org/apache/storm/TestConfigValidate.java    |   726 +
 .../storm/blobstore/ClientBlobStoreTest.java    |   179 +
 .../jvm/org/apache/storm/bolt/TestJoinBolt.java |   357 +
 .../cluster/StormClusterStateImplTest.java      |   116 +
 .../daemon/worker/LogConfigManagerTest.java     |   202 +
 .../apache/storm/daemon/worker/WorkerTest.java  |    39 +
 .../DependencyPropertiesParserTest.java         |    72 +
 .../dependency/DependencyUploaderTest.java      |   296 +
 .../storm/executor/error/ReportErrorTest.java   |    74 +
 .../storm/grouping/PartialKeyGroupingTest.java  |    66 +
 .../storm/grouping/ShuffleGroupingTest.java     |   147 +
 .../metric/filter/FilterByMetricNameTest.java   |    95 +
 .../metric/internal/CountStatAndMetricTest.java |    86 +
 .../internal/LatencyStatAndMetricTest.java      |    83 +
 .../storm/metric/internal/RateTrackerTest.java  |    94 +
 .../metric/util/DataPointExpanderTest.java      |   109 +
 .../storm/security/auth/AuthUtilsTest.java      |   240 +
 .../storm/security/auth/AuthUtilsTestMock.java  |    82 +
 .../apache/storm/security/auth/AutoSSLTest.java |   136 +
 .../auth/DefaultHttpCredentialsPluginTest.java  |    90 +
 .../storm/security/auth/ReqContextTest.java     |    87 +
 .../security/auth/SaslTransportPluginTest.java  |    49 +
 .../storm/security/auth/ThriftClientTest.java   |    68 +
 .../storm/security/auth/ThriftServerTest.java   |    38 +
 .../authorizer/DRPCSimpleACLAuthorizerTest.java |   163 +
 .../BlowfishTupleSerializerTest.java            |    86 +
 .../GzipBridgeSerializationDelegateTest.java    |    82 +
 ...ipBridgeThriftSerializationDelegateTest.java |    71 +
 .../serialization/SerializationFactoryTest.java |    61 +
 .../ThriftBridgeSerializationDelegateTest.java  |    60 +
 .../apache/storm/spout/CheckpointSpoutTest.java |   262 +
 .../storm/state/InMemoryKeyValueStateTest.java  |    99 +
 .../apache/storm/streams/ProcessorBoltTest.java |   174 +
 .../streams/StatefulProcessorBoltTest.java      |   111 +
 .../apache/storm/streams/StreamBuilderTest.java |   268 +
 .../streams/WindowedProcessorBoltTest.java      |   110 +
 .../streams/processors/JoinProcessorTest.java   |   125 +
 .../topology/StatefulBoltExecutorTest.java      |   178 +
 .../StatefulWindowedBoltExecutorTest.java       |   131 +
 .../storm/topology/TopologyBuilderTest.java     |   118 +
 .../topology/WindowedBoltExecutorTest.java      |   230 +
 .../storm/trident/TestTridentTopology.java      |    76 +
 .../storm/trident/TridentWindowingTest.java     |   105 +
 .../jvm/org/apache/storm/tuple/FieldsTest.java  |   126 +
 .../apache/storm/utils/CuratorUtilsTest.java    |   108 +
 .../utils/DisruptorQueueBackpressureTest.java   |   110 +
 .../apache/storm/utils/DisruptorQueueTest.java  |   187 +
 .../apache/storm/utils/MockTupleHelpers.java    |    40 +
 .../storm/utils/ShellBoltMessageQueueTest.java  |    85 +
 ...StormBoundedExponentialBackoffRetryTest.java |   101 +
 .../storm/utils/ThriftTopologyUtilsTest.java    |   106 +
 .../utils/ThrowableNestedCauseMatcher.java      |    44 +
 .../jvm/org/apache/storm/utils/TimeTest.java    |    95 +
 .../jvm/org/apache/storm/utils/UtilsTest.java   |   141 +
 .../utils/WorkerBackpressureThreadTest.java     |    50 +
 .../utils/ZookeeperServerCnxnFactoryTest.java   |    39 +
 .../windowing/WaterMarkEventGeneratorTest.java  |   118 +
 .../storm/windowing/WindowManagerTest.java      |   570 +
 .../drpc-simple-acl-test-scenario.yaml          |    28 +
 storm-clojure/pom.xml                           |     6 +
 .../src/clj/org/apache/storm/testing.clj        |     2 +-
 .../src/clj/org/apache/storm/thrift.clj         |    23 +-
 storm-clojure/src/test/clj/clojure_test.clj     |    40 +-
 storm-core/pom.xml                              |   242 +-
 .../apache/storm/command/shell_submission.clj   |     4 +-
 storm-core/src/clj/org/apache/storm/config.clj  |     6 +-
 .../clj/org/apache/storm/daemon/logviewer.clj   |    68 +-
 .../clj/org/apache/storm/internal/thrift.clj    |    17 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |    84 +-
 storm-core/src/clj/org/apache/storm/util.clj    |    34 +-
 .../src/dev/drpc-simple-acl-test-scenario.yaml  |    28 -
 storm-core/src/genthrift.sh                     |    33 -
 storm-core/src/java_license_header.txt          |    17 -
 storm-core/src/jvm/org/apache/storm/Config.java |  2593 -
 .../src/jvm/org/apache/storm/Constants.java     |    60 -
 .../org/apache/storm/ICredentialsListener.java  |    32 -
 .../src/jvm/org/apache/storm/ILocalCluster.java |   202 -
 .../src/jvm/org/apache/storm/ILocalDRPC.java    |    36 -
 .../jvm/org/apache/storm/ISubmitterHook.java    |    31 -
 .../src/jvm/org/apache/storm/LocalCluster.java  |   834 -
 .../src/jvm/org/apache/storm/LocalDRPC.java     |    90 -
 .../src/jvm/org/apache/storm/LogWriter.java     |    83 -
 .../jvm/org/apache/storm/ProcessSimulator.java  |    97 -
 .../jvm/org/apache/storm/StormSubmitter.java    |   602 -
 .../src/jvm/org/apache/storm/StormTimer.java    |   245 -
 .../src/jvm/org/apache/storm/Testing.java       |   716 -
 storm-core/src/jvm/org/apache/storm/Thrift.java |   351 -
 .../storm/annotation/InterfaceStability.java    |    54 -
 .../storm/blobstore/AtomicOutputStream.java     |    32 -
 .../storm/blobstore/BlobKeySequenceInfo.java    |    40 -
 .../org/apache/storm/blobstore/BlobStore.java   |   490 -
 .../storm/blobstore/BlobStoreAclHandler.java    |   409 -
 .../apache/storm/blobstore/BlobStoreFile.java   |    50 -
 .../apache/storm/blobstore/BlobStoreUtils.java  |   288 -
 .../storm/blobstore/BlobSynchronizer.java       |   131 -
 .../apache/storm/blobstore/ClientBlobStore.java |   201 -
 .../storm/blobstore/FileBlobStoreImpl.java      |   268 -
 .../storm/blobstore/InputStreamWithMeta.java    |    26 -
 .../org/apache/storm/blobstore/KeyFilter.java   |    22 -
 .../storm/blobstore/KeySequenceNumber.java      |   238 -
 .../storm/blobstore/LocalFsBlobStore.java       |   350 -
 .../storm/blobstore/LocalFsBlobStoreFile.java   |   159 -
 .../apache/storm/blobstore/NimbusBlobStore.java |   428 -
 .../src/jvm/org/apache/storm/bolt/JoinBolt.java |   595 -
 .../storm/callback/DefaultWatcherCallBack.java  |    35 -
 .../apache/storm/callback/WatcherCallBack.java  |    24 -
 .../storm/callback/ZKStateChangedCallback.java  |    25 -
 .../org/apache/storm/clojure/ClojureBolt.java   |   119 -
 .../org/apache/storm/clojure/ClojureSpout.java  |   153 -
 .../org/apache/storm/clojure/RichShellBolt.java |    51 -
 .../apache/storm/clojure/RichShellSpout.java    |    51 -
 .../storm/cluster/ClusterStateContext.java      |    41 -
 .../storm/cluster/ClusterStateListener.java     |    22 -
 .../org/apache/storm/cluster/ClusterUtils.java  |   244 -
 .../apache/storm/cluster/ConnectionState.java   |    24 -
 .../org/apache/storm/cluster/DaemonType.java    |    27 -
 .../org/apache/storm/cluster/ExecutorBeat.java  |    44 -
 .../org/apache/storm/cluster/IStateStorage.java |   219 -
 .../storm/cluster/IStormClusterState.java       |   198 -
 .../storm/cluster/PaceMakerStateStorage.java    |   266 -
 .../cluster/PaceMakerStateStorageFactory.java   |    37 -
 .../storm/cluster/StateStorageFactory.java      |    28 -
 .../storm/cluster/StormClusterStateImpl.java    |   724 -
 .../org/apache/storm/cluster/VersionedData.java |    36 -
 .../apache/storm/cluster/ZKStateStorage.java    |   252 -
 .../storm/cluster/ZKStateStorageFactory.java    |    36 -
 .../org/apache/storm/command/AdminCommands.java |    14 +-
 .../jvm/org/apache/storm/command/Blobstore.java |     4 +-
 .../org/apache/storm/command/ConfigValue.java   |     4 +-
 .../org/apache/storm/command/DevZookeeper.java  |    10 +-
 .../jvm/org/apache/storm/command/GetErrors.java |     2 +-
 .../org/apache/storm/command/HealthCheck.java   |   102 +-
 .../org/apache/storm/command/Heartbeats.java    |     4 +-
 .../org/apache/storm/command/KillWorkers.java   |     4 +-
 .../org/apache/storm/command/SetLogLevel.java   |     2 +-
 .../container/ResourceIsolationInterface.java   |    76 -
 .../storm/container/cgroup/CgroupCenter.java    |   216 -
 .../storm/container/cgroup/CgroupCommon.java    |   270 -
 .../container/cgroup/CgroupCommonOperation.java |    81 -
 .../container/cgroup/CgroupCoreFactory.java     |    72 -
 .../storm/container/cgroup/CgroupManager.java   |   223 -
 .../storm/container/cgroup/CgroupOperation.java |    79 -
 .../storm/container/cgroup/CgroupUtils.java     |   118 -
 .../apache/storm/container/cgroup/Device.java   |    75 -
 .../storm/container/cgroup/Hierarchy.java       |   130 -
 .../storm/container/cgroup/SubSystem.java       |    81 -
 .../storm/container/cgroup/SubSystemType.java   |    36 -
 .../storm/container/cgroup/SystemOperation.java |    76 -
 .../storm/container/cgroup/core/BlkioCore.java  |   213 -
 .../storm/container/cgroup/core/CgroupCore.java |    26 -
 .../storm/container/cgroup/core/CpuCore.java    |   135 -
 .../container/cgroup/core/CpuacctCore.java      |    71 -
 .../storm/container/cgroup/core/CpusetCore.java |   209 -
 .../container/cgroup/core/DevicesCore.java      |   189 -
 .../container/cgroup/core/FreezerCore.java      |    66 -
 .../storm/container/cgroup/core/MemoryCore.java |   188 -
 .../storm/container/cgroup/core/NetClsCore.java |    69 -
 .../container/cgroup/core/NetPrioCore.java      |    65 -
 .../storm/coordination/BatchBoltExecutor.java   |   108 -
 .../coordination/BatchOutputCollector.java      |    46 -
 .../coordination/BatchOutputCollectorImpl.java  |    53 -
 .../coordination/BatchSubtopologyBuilder.java   |   447 -
 .../storm/coordination/CoordinatedBolt.java     |   386 -
 .../apache/storm/coordination/IBatchBolt.java   |    30 -
 .../src/jvm/org/apache/storm/daemon/Acker.java  |   140 -
 .../daemon/ClientJarTransformerRunner.java      |    12 +-
 .../org/apache/storm/daemon/DaemonCommon.java   |    22 -
 .../apache/storm/daemon/DirectoryCleaner.java   |     3 -
 .../org/apache/storm/daemon/GrouperFactory.java |   244 -
 .../org/apache/storm/daemon/Shutdownable.java   |    22 -
 .../org/apache/storm/daemon/StormCommon.java    |   596 -
 .../src/jvm/org/apache/storm/daemon/Task.java   |   247 -
 .../daemon/drpc/BlockingOutstandingRequest.java |    67 -
 .../jvm/org/apache/storm/daemon/drpc/DRPC.java  |   215 -
 .../apache/storm/daemon/drpc/DRPCThrift.java    |    58 -
 .../storm/daemon/drpc/OutstandingRequest.java   |    58 -
 .../storm/daemon/drpc/RequestFactory.java       |    24 -
 .../daemon/metrics/BuiltinBoltMetrics.java      |    78 -
 .../storm/daemon/metrics/BuiltinMetrics.java    |    33 -
 .../daemon/metrics/BuiltinMetricsUtil.java      |    81 -
 .../daemon/metrics/BuiltinSpoutMetrics.java     |    64 -
 .../storm/daemon/metrics/MetricsUtils.java      |   108 -
 .../daemon/metrics/SpoutThrottlingMetrics.java  |    57 -
 .../reporters/ConsolePreparableReporter.java    |    76 -
 .../reporters/CsvPreparableReporter.java        |    80 -
 .../reporters/JmxPreparableReporter.java        |    70 -
 .../metrics/reporters/PreparableReporter.java   |    32 -
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |  3805 --
 .../storm/daemon/nimbus/TopologyActions.java    |    31 -
 .../storm/daemon/nimbus/TopologyResources.java  |    63 -
 .../daemon/nimbus/TopologyStateTransition.java  |    27 -
 .../storm/daemon/supervisor/AdvancedFSOps.java  |   363 -
 .../storm/daemon/supervisor/BasicContainer.java |   719 -
 .../supervisor/BasicContainerLauncher.java      |    62 -
 .../storm/daemon/supervisor/Container.java      |   565 -
 .../daemon/supervisor/ContainerLauncher.java    |   104 -
 .../supervisor/ContainerRecoveryException.java  |    29 -
 .../DefaultUncaughtExceptionHandler.java        |    31 -
 .../supervisor/EventManagerPushCallback.java    |    37 -
 .../daemon/supervisor/ExitCodeCallback.java     |    30 -
 .../storm/daemon/supervisor/Killable.java       |    50 -
 .../storm/daemon/supervisor/LocalContainer.java |    94 -
 .../supervisor/LocalContainerLauncher.java      |    60 -
 .../daemon/supervisor/ReadClusterState.java     |   358 -
 .../daemon/supervisor/RunAsUserContainer.java   |   100 -
 .../supervisor/RunAsUserContainerLauncher.java  |    60 -
 .../apache/storm/daemon/supervisor/Slot.java    |   785 -
 .../daemon/supervisor/StandaloneSupervisor.java |    91 -
 .../storm/daemon/supervisor/Supervisor.java     |   359 -
 .../daemon/supervisor/SupervisorUtils.java      |   321 -
 .../apache/storm/daemon/supervisor/UniFunc.java |    22 -
 .../supervisor/timer/SupervisorHealthCheck.java |    41 -
 .../supervisor/timer/SupervisorHeartbeat.java   |    87 -
 .../daemon/supervisor/timer/UpdateBlobs.java    |   105 -
 .../storm/daemon/worker/LogConfigManager.java   |   156 -
 .../org/apache/storm/daemon/worker/Worker.java  |   458 -
 .../apache/storm/daemon/worker/WorkerState.java |   691 -
 .../dependency/DependencyPropertiesParser.java  |    61 -
 .../storm/dependency/DependencyUploader.java    |   168 -
 .../dependency/FileNotAvailableException.java   |    33 -
 .../storm/drpc/DRPCInvocationsClient.java       |   130 -
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |   286 -
 .../jvm/org/apache/storm/drpc/JoinResult.java   |    75 -
 .../org/apache/storm/drpc/KeyedFairBolt.java    |    93 -
 .../storm/drpc/LinearDRPCInputDeclarer.java     |    52 -
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |   393 -
 .../org/apache/storm/drpc/PrepareRequest.java   |    59 -
 .../org/apache/storm/drpc/ReturnResults.java    |   137 -
 .../org/apache/storm/event/EventManager.java    |    24 -
 .../org/apache/storm/event/EventManagerImp.java |   100 -
 .../jvm/org/apache/storm/executor/Executor.java |   563 -
 .../apache/storm/executor/ExecutorShutdown.java |   114 -
 .../apache/storm/executor/ExecutorTransfer.java |    88 -
 .../apache/storm/executor/IRunningExecutor.java |    31 -
 .../apache/storm/executor/LocalExecutor.java    |    56 -
 .../org/apache/storm/executor/TupleInfo.java    |    90 -
 .../storm/executor/bolt/BoltExecutor.java       |   139 -
 .../executor/bolt/BoltOutputCollectorImpl.java  |   171 -
 .../storm/executor/error/IReportError.java      |    22 -
 .../storm/executor/error/ReportError.java       |    76 -
 .../storm/executor/error/ReportErrorAndDie.java |    49 -
 .../storm/executor/spout/SpoutExecutor.java     |   257 -
 .../spout/SpoutOutputCollectorImpl.java         |   147 -
 .../apache/storm/generated/AccessControl.java   |   627 -
 .../storm/generated/AccessControlType.java      |    62 -
 .../storm/generated/AlreadyAliveException.java  |   406 -
 .../org/apache/storm/generated/Assignment.java  |  1159 -
 .../storm/generated/AuthorizationException.java |   406 -
 .../storm/generated/BeginDownloadResult.java    |   608 -
 .../jvm/org/apache/storm/generated/Bolt.java    |   514 -
 .../storm/generated/BoltAggregateStats.java     |   704 -
 .../org/apache/storm/generated/BoltStats.java   |  1390 -
 .../apache/storm/generated/ClusterSummary.java  |   879 -
 .../storm/generated/ClusterWorkerHeartbeat.java |   768 -
 .../storm/generated/CommonAggregateStats.java   |  1060 -
 .../generated/ComponentAggregateStats.java      |   752 -
 .../apache/storm/generated/ComponentCommon.java |   852 -
 .../apache/storm/generated/ComponentObject.java |   462 -
 .../storm/generated/ComponentPageInfo.java      |  2352 -
 .../apache/storm/generated/ComponentType.java   |    62 -
 .../org/apache/storm/generated/Credentials.java |   458 -
 .../storm/generated/DRPCExceptionType.java      |    68 -
 .../storm/generated/DRPCExecutionException.java |   528 -
 .../org/apache/storm/generated/DRPCRequest.java |   507 -
 .../apache/storm/generated/DebugOptions.java    |   506 -
 .../apache/storm/generated/DistributedRPC.java  |  1328 -
 .../generated/DistributedRPCInvocations.java    |  3904 --
 .../org/apache/storm/generated/ErrorInfo.java   |   714 -
 .../storm/generated/ExecutorAggregateStats.java |   526 -
 .../apache/storm/generated/ExecutorInfo.java    |   499 -
 .../storm/generated/ExecutorSpecificStats.java  |   387 -
 .../apache/storm/generated/ExecutorStats.java   |   915 -
 .../apache/storm/generated/ExecutorSummary.java |   922 -
 .../apache/storm/generated/GetInfoOptions.java  |   422 -
 .../apache/storm/generated/GlobalStreamId.java  |   507 -
 .../org/apache/storm/generated/Grouping.java    |   800 -
 .../generated/HBAuthorizationException.java     |   406 -
 .../storm/generated/HBExecutionException.java   |   406 -
 .../org/apache/storm/generated/HBMessage.java   |   636 -
 .../apache/storm/generated/HBMessageData.java   |   640 -
 .../jvm/org/apache/storm/generated/HBNodes.java |   461 -
 .../jvm/org/apache/storm/generated/HBPulse.java |   522 -
 .../org/apache/storm/generated/HBRecords.java   |   466 -
 .../storm/generated/HBServerMessageType.java    |   113 -
 .../generated/InvalidTopologyException.java     |   406 -
 .../org/apache/storm/generated/JavaObject.java  |   561 -
 .../apache/storm/generated/JavaObjectArg.java   |   631 -
 .../generated/KeyAlreadyExistsException.java    |   406 -
 .../storm/generated/KeyNotFoundException.java   |   406 -
 .../org/apache/storm/generated/KillOptions.java |   407 -
 .../storm/generated/LSApprovedWorkers.java      |   458 -
 .../generated/LSSupervisorAssignments.java      |   471 -
 .../apache/storm/generated/LSSupervisorId.java  |   406 -
 .../apache/storm/generated/LSTopoHistory.java   |   805 -
 .../storm/generated/LSTopoHistoryList.java      |   460 -
 .../storm/generated/LSWorkerHeartbeat.java      |   755 -
 .../apache/storm/generated/ListBlobsResult.java |   556 -
 .../apache/storm/generated/LocalAssignment.java |   676 -
 .../apache/storm/generated/LocalStateData.java  |   471 -
 .../org/apache/storm/generated/LogConfig.java   |   475 -
 .../org/apache/storm/generated/LogLevel.java    |   836 -
 .../apache/storm/generated/LogLevelAction.java  |    65 -
 .../jvm/org/apache/storm/generated/Nimbus.java  | 47241 -----------------
 .../apache/storm/generated/NimbusSummary.java   |   796 -
 .../org/apache/storm/generated/NodeInfo.java    |   556 -
 .../storm/generated/NotAliveException.java      |   406 -
 .../org/apache/storm/generated/NullStruct.java  |   300 -
 .../apache/storm/generated/NumErrorsChoice.java |    65 -
 .../apache/storm/generated/ProfileAction.java   |    74 -
 .../apache/storm/generated/ProfileRequest.java  |   631 -
 .../storm/generated/ReadableBlobMeta.java       |   510 -
 .../storm/generated/RebalanceOptions.java       |   664 -
 .../storm/generated/SettableBlobMeta.java       |   567 -
 .../apache/storm/generated/ShellComponent.java  |   516 -
 .../storm/generated/SpecificAggregateStats.java |   387 -
 .../storm/generated/SpoutAggregateStats.java    |   407 -
 .../org/apache/storm/generated/SpoutSpec.java   |   514 -
 .../org/apache/storm/generated/SpoutStats.java  |   917 -
 .../apache/storm/generated/StateSpoutSpec.java  |   514 -
 .../org/apache/storm/generated/StormBase.java   |  1382 -
 .../apache/storm/generated/StormTopology.java   |  1272 -
 .../org/apache/storm/generated/StreamInfo.java  |   554 -
 .../apache/storm/generated/SubmitOptions.java   |   533 -
 .../apache/storm/generated/SupervisorInfo.java  |  1446 -
 .../storm/generated/SupervisorPageInfo.java     |   624 -
 .../storm/generated/SupervisorSummary.java      |  1265 -
 .../storm/generated/ThriftSerializedObject.java |   516 -
 .../storm/generated/TopologyActionOptions.java  |   387 -
 .../storm/generated/TopologyHistoryInfo.java    |   461 -
 .../apache/storm/generated/TopologyInfo.java    |  2144 -
 .../storm/generated/TopologyInitialStatus.java  |    62 -
 .../storm/generated/TopologyPageInfo.java       |  2757 -
 .../apache/storm/generated/TopologyStats.java   |  1094 -
 .../apache/storm/generated/TopologyStatus.java  |    68 -
 .../apache/storm/generated/TopologySummary.java |  1901 -
 .../apache/storm/generated/WorkerResources.java |   605 -
 .../apache/storm/generated/WorkerSummary.java   |  1880 -
 .../storm/grouping/CustomStreamGrouping.java    |    43 -
 .../src/jvm/org/apache/storm/grouping/Load.java |    77 -
 .../grouping/LoadAwareCustomStreamGrouping.java |    24 -
 .../grouping/LoadAwareShuffleGrouping.java      |    76 -
 .../org/apache/storm/grouping/LoadMapping.java  |    64 -
 .../storm/grouping/PartialKeyGrouping.java      |   106 -
 .../apache/storm/grouping/ShuffleGrouping.java  |    63 -
 .../org/apache/storm/hooks/BaseTaskHook.java    |    61 -
 .../org/apache/storm/hooks/BaseWorkerHook.java  |    51 -
 .../jvm/org/apache/storm/hooks/ITaskHook.java   |    38 -
 .../jvm/org/apache/storm/hooks/IWorkerHook.java |    44 -
 .../storm/hooks/SubmitterHookException.java     |    41 -
 .../apache/storm/hooks/info/BoltAckInfo.java    |    40 -
 .../storm/hooks/info/BoltExecuteInfo.java       |    40 -
 .../apache/storm/hooks/info/BoltFailInfo.java   |    40 -
 .../org/apache/storm/hooks/info/EmitInfo.java   |    44 -
 .../apache/storm/hooks/info/SpoutAckInfo.java   |    39 -
 .../apache/storm/hooks/info/SpoutFailInfo.java  |    39 -
 .../apache/storm/localizer/AsyncLocalizer.java  |   436 -
 .../org/apache/storm/localizer/ILocalizer.java  |    70 -
 .../localizer/LocalDownloadedResource.java      |   146 -
 .../apache/storm/localizer/LocalResource.java   |    44 -
 .../storm/localizer/LocalizedResource.java      |   130 -
 .../LocalizedResourceRetentionSet.java          |   140 -
 .../storm/localizer/LocalizedResourceSet.java   |   101 -
 .../org/apache/storm/localizer/Localizer.java   |   702 -
 .../storm/logging/ThriftAccessLogger.java       |    56 -
 .../logging/filters/AccessLoggingFilter.java    |    52 -
 .../storm/messaging/ConnectionWithStatus.java   |    49 -
 .../DeserializingConnectionCallback.java        |    60 -
 .../org/apache/storm/messaging/IConnection.java |    63 -
 .../storm/messaging/IConnectionCallback.java    |    31 -
 .../org/apache/storm/messaging/IContext.java    |    59 -
 .../org/apache/storm/messaging/TaskMessage.java |    53 -
 .../storm/messaging/TransportFactory.java       |    57 -
 .../apache/storm/messaging/local/Context.java   |   218 -
 .../apache/storm/messaging/netty/Client.java    |   612 -
 .../apache/storm/messaging/netty/Context.java   |   113 -
 .../storm/messaging/netty/ControlMessage.java   |    75 -
 .../messaging/netty/INettySerializable.java     |    26 -
 .../storm/messaging/netty/ISaslClient.java      |    28 -
 .../storm/messaging/netty/ISaslServer.java      |    26 -
 .../apache/storm/messaging/netty/IServer.java   |    26 -
 .../netty/KerberosSaslClientHandler.java        |   154 -
 .../netty/KerberosSaslNettyClient.java          |   202 -
 .../netty/KerberosSaslNettyClientState.java     |    31 -
 .../netty/KerberosSaslNettyServer.java          |   209 -
 .../netty/KerberosSaslNettyServerState.java     |    30 -
 .../netty/KerberosSaslServerHandler.java        |   133 -
 .../org/apache/storm/messaging/netty/Login.java |   411 -
 .../storm/messaging/netty/MessageBatch.java     |   118 -
 .../storm/messaging/netty/MessageBuffer.java    |    58 -
 .../storm/messaging/netty/MessageDecoder.java   |   144 -
 .../storm/messaging/netty/MessageEncoder.java   |    43 -
 .../netty/NettyRenameThreadFactory.java         |    56 -
 .../netty/NettyUncaughtExceptionHandler.java    |    35 -
 .../storm/messaging/netty/SaslMessageToken.java |   114 -
 .../storm/messaging/netty/SaslNettyClient.java  |   154 -
 .../messaging/netty/SaslNettyClientState.java   |    31 -
 .../storm/messaging/netty/SaslNettyServer.java  |   157 -
 .../messaging/netty/SaslNettyServerState.java   |    30 -
 .../messaging/netty/SaslStormClientHandler.java |   158 -
 .../netty/SaslStormServerAuthorizeHandler.java  |    83 -
 .../messaging/netty/SaslStormServerHandler.java |   153 -
 .../apache/storm/messaging/netty/SaslUtils.java |    68 -
 .../apache/storm/messaging/netty/Server.java    |   273 -
 .../messaging/netty/StormClientHandler.java     |    90 -
 .../netty/StormClientPipelineFactory.java       |    56 -
 .../messaging/netty/StormServerHandler.java     |    74 -
 .../netty/StormServerPipelineFactory.java       |    57 -
 .../metric/ClusterMetricsConsumerExecutor.java  |    89 -
 .../apache/storm/metric/EventLoggerBolt.java    |    67 -
 .../storm/metric/FileBasedEventLogger.java      |   122 -
 .../metric/HttpForwardingMetricsConsumer.java   |    85 -
 .../metric/HttpForwardingMetricsServer.java     |   118 -
 .../org/apache/storm/metric/IEventLogger.java   |    70 -
 .../metric/LoggingClusterMetricsConsumer.java   |    86 -
 .../storm/metric/LoggingMetricsConsumer.java    |    76 -
 .../storm/metric/MetricsConsumerBolt.java       |   146 -
 .../storm/metric/StormMetricsRegistry.java      |    88 -
 .../jvm/org/apache/storm/metric/SystemBolt.java |   160 -
 .../storm/metric/api/AssignableMetric.java      |    34 -
 .../apache/storm/metric/api/CombinedMetric.java |    38 -
 .../apache/storm/metric/api/CountMetric.java    |    39 -
 .../org/apache/storm/metric/api/DataPoint.java  |    43 -
 .../metric/api/IClusterMetricsConsumer.java     |    63 -
 .../org/apache/storm/metric/api/ICombiner.java  |    23 -
 .../org/apache/storm/metric/api/IMetric.java    |    30 -
 .../storm/metric/api/IMetricsConsumer.java      |    81 -
 .../org/apache/storm/metric/api/IReducer.java   |    24 -
 .../storm/metric/api/IStatefulObject.java       |    22 -
 .../apache/storm/metric/api/MeanReducer.java    |    53 -
 .../storm/metric/api/MultiCountMetric.java      |    45 -
 .../storm/metric/api/MultiReducedMetric.java    |    50 -
 .../apache/storm/metric/api/ReducedMetric.java  |    38 -
 .../apache/storm/metric/api/StateMetric.java    |    31 -
 .../metric/api/rpc/AssignableShellMetric.java   |    30 -
 .../metric/api/rpc/CombinedShellMetric.java     |    31 -
 .../storm/metric/api/rpc/CountShellMetric.java  |    37 -
 .../storm/metric/api/rpc/IShellMetric.java      |    31 -
 .../metric/api/rpc/ReducedShellMetric.java      |    32 -
 .../apache/storm/metric/cgroup/CGroupCpu.java   |    70 -
 .../storm/metric/cgroup/CGroupCpuGuarantee.java |    52 -
 .../storm/metric/cgroup/CGroupMemoryLimit.java  |    39 -
 .../storm/metric/cgroup/CGroupMemoryUsage.java  |    39 -
 .../storm/metric/cgroup/CGroupMetricsBase.java  |   114 -
 .../storm/metric/filter/FilterByMetricName.java |   110 -
 .../storm/metric/filter/MetricsFilter.java      |    26 -
 .../metric/internal/CountStatAndMetric.java     |   211 -
 .../metric/internal/LatencyStatAndMetric.java   |   262 -
 .../storm/metric/internal/MetricStatTimer.java  |    27 -
 .../internal/MultiCountStatAndMetric.java       |   112 -
 .../internal/MultiLatencyStatAndMetric.java     |   109 -
 .../storm/metric/internal/RateTracker.java      |   139 -
 .../storm/metric/util/DataPointExpander.java    |    79 -
 .../jvm/org/apache/storm/multilang/BoltMsg.java |    79 -
 .../org/apache/storm/multilang/ISerializer.java |    82 -
 .../apache/storm/multilang/JsonSerializer.java  |   203 -
 .../storm/multilang/NoOutputException.java      |    40 -
 .../org/apache/storm/multilang/ShellMsg.java    |   184 -
 .../org/apache/storm/multilang/SpoutMsg.java    |    50 -
 .../AbstractDNSToSwitchMapping.java             |    96 -
 .../networktopography/DNSToSwitchMapping.java   |    49 -
 .../DefaultRackDNSToSwitchMapping.java          |    52 -
 .../storm/nimbus/DefaultTopologyValidator.java  |    31 -
 .../org/apache/storm/nimbus/ILeaderElector.java |    73 -
 .../nimbus/ITopologyActionNotifierPlugin.java   |    43 -
 .../apache/storm/nimbus/ITopologyValidator.java |    28 -
 .../jvm/org/apache/storm/nimbus/NimbusInfo.java |   120 -
 .../storm/pacemaker/IServerMessageHandler.java  |    25 -
 .../org/apache/storm/pacemaker/Pacemaker.java   |   218 -
 .../apache/storm/pacemaker/PacemakerClient.java |   274 -
 .../storm/pacemaker/PacemakerClientHandler.java |    80 -
 .../storm/pacemaker/PacemakerClientPool.java    |   113 -
 .../pacemaker/PacemakerConnectionException.java |    24 -
 .../apache/storm/pacemaker/PacemakerServer.java |   168 -
 .../storm/pacemaker/codec/ThriftDecoder.java    |    69 -
 .../storm/pacemaker/codec/ThriftEncoder.java    |   110 -
 .../pacemaker/codec/ThriftNettyClientCodec.java |    97 -
 .../pacemaker/codec/ThriftNettyServerCodec.java |    99 -
 .../jvm/org/apache/storm/scheduler/Cluster.java |   855 -
 .../storm/scheduler/DefaultScheduler.java       |   111 -
 .../apache/storm/scheduler/EvenScheduler.java   |   171 -
 .../apache/storm/scheduler/ExecutorDetails.java |    54 -
 .../jvm/org/apache/storm/scheduler/INimbus.java |    49 -
 .../org/apache/storm/scheduler/IScheduler.java  |    40 -
 .../org/apache/storm/scheduler/ISupervisor.java |    45 -
 .../storm/scheduler/IsolationScheduler.java     |   425 -
 .../storm/scheduler/SchedulerAssignment.java    |    61 -
 .../scheduler/SchedulerAssignmentImpl.java      |   135 -
 .../storm/scheduler/SupervisorDetails.java      |   138 -
 .../org/apache/storm/scheduler/Topologies.java  |    87 -
 .../apache/storm/scheduler/TopologyDetails.java |   516 -
 .../org/apache/storm/scheduler/WorkerSlot.java  |    81 -
 .../scheduler/multitenant/DefaultPool.java      |   219 -
 .../storm/scheduler/multitenant/FreePool.java   |   125 -
 .../scheduler/multitenant/IsolatedPool.java     |   371 -
 .../multitenant/MultitenantScheduler.java       |    98 -
 .../storm/scheduler/multitenant/Node.java       |   340 -
 .../storm/scheduler/multitenant/NodePool.java   |   296 -
 .../storm/scheduler/resource/Component.java     |    54 -
 .../storm/scheduler/resource/RAS_Node.java      |   529 -
 .../storm/scheduler/resource/RAS_Nodes.java     |   138 -
 .../resource/ResourceAwareScheduler.java        |   435 -
 .../storm/scheduler/resource/ResourceUtils.java |   206 -
 .../scheduler/resource/SchedulingResult.java    |   116 -
 .../scheduler/resource/SchedulingState.java     |    56 -
 .../scheduler/resource/SchedulingStatus.java    |    40 -
 .../apache/storm/scheduler/resource/User.java   |   350 -
 .../eviction/DefaultEvictionStrategy.java       |   126 -
 .../strategies/eviction/IEvictionStrategy.java  |    42 -
 .../DefaultSchedulingPriorityStrategy.java      |    80 -
 .../priority/ISchedulingPriorityStrategy.java   |    36 -
 .../DefaultResourceAwareStrategy.java           |   757 -
 .../strategies/scheduling/IStrategy.java        |    47 -
 .../storm/security/INimbusCredentialPlugin.java |    47 -
 .../auth/AbstractSaslClientCallbackHandler.java |    76 -
 .../auth/AbstractSaslServerCallbackHandler.java |    94 -
 .../apache/storm/security/auth/AuthUtils.java   |   419 -
 .../org/apache/storm/security/auth/AutoSSL.java |   161 -
 .../auth/DefaultHttpCredentialsPlugin.java      |    96 -
 .../security/auth/DefaultPrincipalToLocal.java  |    42 -
 .../apache/storm/security/auth/IAuthorizer.java |    53 -
 .../storm/security/auth/IAutoCredentials.java   |    55 -
 .../security/auth/ICredentialsRenewer.java      |    40 -
 .../auth/IGroupMappingServiceProvider.java      |    42 -
 .../security/auth/IHttpCredentialsPlugin.java   |    48 -
 .../storm/security/auth/IPrincipalToLocal.java  |    41 -
 .../storm/security/auth/ITransportPlugin.java   |    57 -
 .../security/auth/KerberosPrincipalToLocal.java |    45 -
 .../storm/security/auth/NimbusPrincipal.java    |    29 -
 .../apache/storm/security/auth/ReqContext.java  |   152 -
 .../security/auth/SaslTransportPlugin.java      |   180 -
 .../security/auth/ShellBasedGroupsMapping.java  |    99 -
 .../security/auth/SimpleTransportPlugin.java    |   163 -
 .../security/auth/SingleUserPrincipal.java      |    53 -
 .../storm/security/auth/TBackoffConnect.java    |    82 -
 .../storm/security/auth/ThriftClient.java       |   124 -
 .../security/auth/ThriftConnectionType.java     |    86 -
 .../storm/security/auth/ThriftServer.java       |    78 -
 .../auth/authorizer/DRPCAuthorizerBase.java     |    63 -
 .../authorizer/DRPCSimpleACLAuthorizer.java     |   177 -
 .../auth/authorizer/DenyAuthorizer.java         |    47 -
 .../authorizer/ImpersonationAuthorizer.java     |   172 -
 .../auth/authorizer/NoopAuthorizer.java         |    47 -
 .../auth/authorizer/SimpleACLAuthorizer.java    |   173 -
 .../authorizer/SimpleWhitelistAuthorizer.java   |    60 -
 .../auth/digest/ClientCallbackHandler.java      |    58 -
 .../auth/digest/DigestSaslTransportPlugin.java  |    68 -
 .../auth/digest/ServerCallbackHandler.java      |    85 -
 .../storm/security/auth/kerberos/AutoTGT.java   |   269 -
 .../auth/kerberos/AutoTGTKrb5LoginModule.java   |   112 -
 .../kerberos/AutoTGTKrb5LoginModuleTest.java    |    44 -
 .../auth/kerberos/ClientCallbackHandler.java    |   104 -
 .../kerberos/KerberosSaslTransportPlugin.java   |   262 -
 .../security/auth/kerberos/NoOpTTrasport.java   |    54 -
 .../auth/kerberos/ServerCallbackHandler.java    |    96 -
 .../auth/kerberos/jaas_kerberos_cluster.conf    |    49 -
 .../auth/kerberos/jaas_kerberos_launcher.conf   |    31 -
 .../auth/plain/PlainClientCallbackHandler.java  |    31 -
 .../auth/plain/PlainSaslTransportPlugin.java    |    71 -
 .../auth/plain/PlainServerCallbackHandler.java  |    55 -
 .../security/auth/plain/SaslPlainServer.java    |   158 -
 .../serialization/BlowfishTupleSerializer.java  |    92 -
 .../storm/serialization/DefaultKryoFactory.java |    64 -
 .../DefaultSerializationDelegate.java           |    58 -
 .../GzipBridgeSerializationDelegate.java        |    65 -
 .../GzipBridgeThriftSerializationDelegate.java  |    64 -
 .../GzipSerializationDelegate.java              |    64 -
 .../GzipThriftSerializationDelegate.java        |    56 -
 .../storm/serialization/IKryoDecorator.java     |    23 -
 .../storm/serialization/IKryoFactory.java       |    40 -
 .../storm/serialization/ITupleDeserializer.java |    24 -
 .../storm/serialization/ITupleSerializer.java   |    26 -
 .../serialization/KryoTupleDeserializer.java    |    56 -
 .../serialization/KryoTupleSerializer.java      |    60 -
 .../serialization/KryoValuesDeserializer.java   |    50 -
 .../serialization/KryoValuesSerializer.java     |    58 -
 .../serialization/SerializableSerializer.java   |    61 -
 .../serialization/SerializationDelegate.java    |    35 -
 .../serialization/SerializationFactory.java     |   240 -
 .../ThriftSerializationDelegate.java            |    52 -
 .../types/ArrayListSerializer.java              |    32 -
 .../serialization/types/HashMapSerializer.java  |    32 -
 .../serialization/types/HashSetSerializer.java  |    32 -
 .../types/ListDelegateSerializer.java           |    32 -
 .../org/apache/storm/spout/CheckPointState.java |   172 -
 .../org/apache/storm/spout/CheckpointSpout.java |   232 -
 .../storm/spout/IMultiSchemableSpout.java       |    23 -
 .../org/apache/storm/spout/ISchemableSpout.java |    24 -
 .../src/jvm/org/apache/storm/spout/ISpout.java  |   105 -
 .../storm/spout/ISpoutOutputCollector.java      |    32 -
 .../apache/storm/spout/ISpoutWaitStrategy.java  |    34 -
 .../jvm/org/apache/storm/spout/MultiScheme.java |    29 -
 .../storm/spout/NothingEmptyEmitStrategy.java   |    31 -
 .../org/apache/storm/spout/RawMultiScheme.java  |    40 -
 .../jvm/org/apache/storm/spout/RawScheme.java   |    37 -
 .../src/jvm/org/apache/storm/spout/Scheme.java  |    29 -
 .../apache/storm/spout/SchemeAsMultiScheme.java |    42 -
 .../jvm/org/apache/storm/spout/ShellSpout.java  |   327 -
 .../storm/spout/SleepSpoutWaitStrategy.java     |    41 -
 .../storm/spout/SpoutOutputCollector.java       |   143 -
 .../storm/state/DefaultStateSerializer.java     |    77 -
 .../jvm/org/apache/storm/state/IStateSpout.java |    29 -
 .../storm/state/IStateSpoutOutputCollector.java |    22 -
 .../apache/storm/state/ISubscribedState.java    |    25 -
 .../state/ISynchronizeOutputCollector.java      |    24 -
 .../storm/state/InMemoryKeyValueState.java      |   126 -
 .../state/InMemoryKeyValueStateProvider.java    |    43 -
 .../org/apache/storm/state/KeyValueState.java   |    58 -
 .../jvm/org/apache/storm/state/Serializer.java  |    30 -
 .../src/jvm/org/apache/storm/state/State.java   |    55 -
 .../org/apache/storm/state/StateFactory.java    |    69 -
 .../org/apache/storm/state/StateProvider.java   |    38 -
 .../storm/state/StateSpoutOutputCollector.java  |    28 -
 .../storm/state/SynchronizeOutputCollector.java |    30 -
 .../apache/storm/stats/BoltExecutorStats.java   |   106 -
 .../jvm/org/apache/storm/stats/CommonStats.java |   114 -
 .../apache/storm/stats/SpoutExecutorStats.java  |    76 -
 .../jvm/org/apache/storm/stats/StatsUtil.java   |  2612 -
 .../src/jvm/org/apache/storm/streams/Edge.java  |    41 -
 .../org/apache/storm/streams/GroupingInfo.java  |   100 -
 .../src/jvm/org/apache/storm/streams/Node.java  |   170 -
 .../src/jvm/org/apache/storm/streams/Pair.java  |   103 -
 .../org/apache/storm/streams/PairStream.java    |   525 -
 .../org/apache/storm/streams/PartitionNode.java |    37 -
 .../org/apache/storm/streams/ProcessorBolt.java |    70 -
 .../storm/streams/ProcessorBoltDelegate.java    |   348 -
 .../org/apache/storm/streams/ProcessorNode.java |    83 -
 .../apache/storm/streams/RefCountedTuple.java   |    64 -
 .../jvm/org/apache/storm/streams/SinkNode.java  |    44 -
 .../jvm/org/apache/storm/streams/SpoutNode.java |    48 -
 .../storm/streams/StatefulProcessorBolt.java    |   116 -
 .../jvm/org/apache/storm/streams/Stream.java    |   515 -
 .../org/apache/storm/streams/StreamBolt.java    |    38 -
 .../org/apache/storm/streams/StreamBuilder.java |   602 -
 .../org/apache/storm/streams/StreamState.java   |    43 -
 .../org/apache/storm/streams/StreamUtil.java    |    69 -
 .../storm/streams/StreamsEdgeFactory.java       |    29 -
 .../org/apache/storm/streams/UniqueIdGen.java   |    56 -
 .../org/apache/storm/streams/WindowNode.java    |    39 -
 .../storm/streams/WindowedProcessorBolt.java    |   137 -
 .../storm/streams/operations/BiFunction.java    |    37 -
 .../streams/operations/CombinerAggregator.java  |    97 -
 .../storm/streams/operations/Consumer.java      |    32 -
 .../streams/operations/FlatMapFunction.java     |    27 -
 .../storm/streams/operations/Function.java      |    34 -
 .../streams/operations/IdentityFunction.java    |    31 -
 .../storm/streams/operations/Operation.java     |    26 -
 .../streams/operations/PairFlatMapFunction.java |    30 -
 .../storm/streams/operations/PairFunction.java  |    30 -
 .../streams/operations/PairValueJoiner.java     |    40 -
 .../storm/streams/operations/Predicate.java     |    33 -
 .../storm/streams/operations/PrintConsumer.java |    30 -
 .../storm/streams/operations/Reducer.java       |    35 -
 .../storm/streams/operations/StateUpdater.java  |    67 -
 .../storm/streams/operations/ValueJoiner.java   |    36 -
 .../streams/operations/aggregators/Count.java   |    47 -
 .../streams/operations/aggregators/LongSum.java |    45 -
 .../operations/mappers/PairValueMapper.java     |    51 -
 .../operations/mappers/TupleValueMapper.java    |    30 -
 .../operations/mappers/TupleValueMappers.java   |   174 -
 .../streams/operations/mappers/ValueMapper.java |    45 -
 .../operations/mappers/ValuesMapper.java        |    48 -
 .../processors/AggregateByKeyProcessor.java     |    78 -
 .../streams/processors/AggregateProcessor.java  |    69 -
 .../storm/streams/processors/BaseProcessor.java |   109 -
 .../streams/processors/BatchProcessor.java      |    25 -
 .../streams/processors/BranchProcessor.java     |    41 -
 .../processors/ChainedProcessorContext.java     |    66 -
 .../processors/EmittingProcessorContext.java    |   155 -
 .../streams/processors/FilterProcessor.java     |    35 -
 .../streams/processors/FlatMapProcessor.java    |    35 -
 .../processors/FlatMapValuesProcessor.java      |    36 -
 .../streams/processors/ForEachProcessor.java    |    33 -
 .../processors/ForwardingProcessorContext.java  |   102 -
 .../storm/streams/processors/JoinProcessor.java |   152 -
 .../storm/streams/processors/MapProcessor.java  |    33 -
 .../streams/processors/MapValuesProcessor.java  |    34 -
 .../MergeAggregateByKeyProcessor.java           |    54 -
 .../processors/MergeAggregateProcessor.java     |    47 -
 .../storm/streams/processors/PeekProcessor.java |    34 -
 .../storm/streams/processors/Processor.java     |    54 -
 .../streams/processors/ProcessorContext.java    |    62 -
 .../processors/ReduceByKeyProcessor.java        |    52 -
 .../streams/processors/ReduceProcessor.java     |    43 -
 .../streams/processors/StateQueryProcessor.java |    48 -
 .../streams/processors/StatefulProcessor.java   |    36 -
 .../processors/UpdateStateByKeyProcessor.java   |    49 -
 .../org/apache/storm/streams/tuple/Tuple10.java |   112 -
 .../org/apache/storm/streams/tuple/Tuple3.java  |    70 -
 .../org/apache/storm/streams/tuple/Tuple4.java  |    76 -
 .../org/apache/storm/streams/tuple/Tuple5.java  |    82 -
 .../org/apache/storm/streams/tuple/Tuple6.java  |    89 -
 .../org/apache/storm/streams/tuple/Tuple7.java  |    94 -
 .../org/apache/storm/streams/tuple/Tuple8.java  |   100 -
 .../org/apache/storm/streams/tuple/Tuple9.java  |   106 -
 .../storm/streams/windowing/BaseWindow.java     |    64 -
 .../storm/streams/windowing/SlidingWindows.java |   151 -
 .../streams/windowing/TumblingWindows.java      |   119 -
 .../apache/storm/streams/windowing/Window.java  |    70 -
 .../storm/task/GeneralTopologyContext.java      |   202 -
 .../src/jvm/org/apache/storm/task/IBolt.java    |    84 -
 .../org/apache/storm/task/IErrorReporter.java   |    22 -
 .../org/apache/storm/task/IMetricsContext.java  |    31 -
 .../org/apache/storm/task/IOutputCollector.java |    33 -
 .../org/apache/storm/task/OutputCollector.java  |   236 -
 .../jvm/org/apache/storm/task/ShellBolt.java    |   435 -
 .../org/apache/storm/task/TopologyContext.java  |   402 -
 .../storm/task/WorkerTopologyContext.java       |   106 -
 .../apache/storm/testing/AckFailDelegate.java   |    25 -
 .../apache/storm/testing/AckFailMapTracker.java |    52 -
 .../org/apache/storm/testing/AckTracker.java    |    52 -
 .../AlternateRackDNSToSwitchMapping.java        |    65 -
 .../apache/storm/testing/BatchNumberList.java   |    72 -
 .../apache/storm/testing/BatchProcessWord.java  |    39 -
 .../org/apache/storm/testing/BatchRepeatA.java  |    48 -
 .../org/apache/storm/testing/BoltTracker.java   |    42 -
 .../apache/storm/testing/CompletableSpout.java  |    39 -
 .../storm/testing/CompleteTopologyParam.java    |   103 -
 .../apache/storm/testing/CountingBatchBolt.java |    55 -
 .../storm/testing/CountingCommitBolt.java       |    57 -
 .../org/apache/storm/testing/FeederSpout.java   |   107 -
 .../org/apache/storm/testing/FixedTuple.java    |    42 -
 .../apache/storm/testing/FixedTupleSpout.java   |   184 -
 .../testing/ForwardingMetricsConsumer.java      |   100 -
 .../org/apache/storm/testing/IdentityBolt.java  |    42 -
 .../storm/testing/InProcessZookeeper.java       |    59 -
 .../apache/storm/testing/IntegrationTest.java   |    38 -
 .../storm/testing/KeyedCountingBatchBolt.java   |    61 -
 .../testing/KeyedCountingCommitterBolt.java     |    24 -
 .../storm/testing/KeyedSummingBatchBolt.java    |    61 -
 .../storm/testing/MemoryTransactionalSpout.java |   180 -
 .../testing/MemoryTransactionalSpoutMeta.java   |    38 -
 .../apache/storm/testing/MkClusterParam.java    |    69 -
 .../org/apache/storm/testing/MkTupleParam.java  |    58 -
 .../org/apache/storm/testing/MockedSources.java |    68 -
 .../jvm/org/apache/storm/testing/NGrouping.java |    48 -
 .../storm/testing/NonRichBoltTracker.java       |    51 -
 .../testing/OpaqueMemoryTransactionalSpout.java |   186 -
 .../apache/storm/testing/PrepareBatchBolt.java  |    52 -
 .../storm/testing/PythonShellMetricsBolt.java   |    53 -
 .../storm/testing/PythonShellMetricsSpout.java  |    56 -
 .../testing/SingleUserSimpleTransport.java      |    37 -
 .../org/apache/storm/testing/SpoutTracker.java  |   111 -
 .../storm/testing/TestAggregatesCounter.java    |    63 -
 .../org/apache/storm/testing/TestConfBolt.java  |    62 -
 .../apache/storm/testing/TestEventLogSpout.java |   144 -
 .../storm/testing/TestEventOrderCheckBolt.java  |    76 -
 .../apache/storm/testing/TestGlobalCount.java   |    60 -
 .../jvm/org/apache/storm/testing/TestJob.java   |    43 -
 .../apache/storm/testing/TestKryoDecorator.java |    31 -
 .../apache/storm/testing/TestPlannerBolt.java   |    45 -
 .../apache/storm/testing/TestPlannerSpout.java  |    85 -
 .../org/apache/storm/testing/TestSerObject.java |    37 -
 .../storm/testing/TestWordBytesCounter.java     |    27 -
 .../apache/storm/testing/TestWordCounter.java   |    65 -
 .../org/apache/storm/testing/TestWordSpout.java |    86 -
 .../jvm/org/apache/storm/testing/TmpPath.java   |    69 -
 .../apache/storm/testing/TrackedTopology.java   |   135 -
 .../apache/storm/testing/TupleCaptureBolt.java  |    83 -
 .../staticmocking/MockedClientZookeeper.java    |    32 +
 .../topology/BaseConfigurationDeclarer.java     |    84 -
 .../topology/BaseStatefulBoltExecutor.java      |   209 -
 .../storm/topology/BasicBoltExecutor.java       |    67 -
 .../storm/topology/BasicOutputCollector.java    |    72 -
 .../org/apache/storm/topology/BoltDeclarer.java |    26 -
 .../topology/CheckpointTupleForwarder.java      |   103 -
 .../ComponentConfigurationDeclarer.java         |    29 -
 .../storm/topology/ConfigurableTopology.java    |   184 -
 .../apache/storm/topology/FailedException.java  |    36 -
 .../org/apache/storm/topology/IBasicBolt.java   |    33 -
 .../storm/topology/IBasicOutputCollector.java   |    29 -
 .../org/apache/storm/topology/IComponent.java   |    44 -
 .../org/apache/storm/topology/IRichBolt.java    |    29 -
 .../org/apache/storm/topology/IRichSpout.java   |    29 -
 .../apache/storm/topology/IRichStateSpout.java  |    25 -
 .../apache/storm/topology/IStatefulBolt.java    |    48 -
 .../storm/topology/IStatefulComponent.java      |    56 -
 .../storm/topology/IStatefulWindowedBolt.java   |    26 -
 .../apache/storm/topology/IWindowedBolt.java    |    51 -
 .../apache/storm/topology/InputDeclarer.java    |   184 -
 .../storm/topology/OutputFieldsDeclarer.java    |    32 -
 .../storm/topology/OutputFieldsGetter.java      |    53 -
 .../storm/topology/ReportedFailedException.java |    36 -
 .../apache/storm/topology/ResourceDeclarer.java |    28 -
 .../apache/storm/topology/SpoutDeclarer.java    |    22 -
 .../storm/topology/StatefulBoltExecutor.java    |   193 -
 .../topology/StatefulWindowedBoltExecutor.java  |   372 -
 .../apache/storm/topology/TopologyBuilder.java  |   588 -
 .../topology/TupleFieldTimestampExtractor.java  |    48 -
 .../storm/topology/WindowedBoltExecutor.java    |   363 -
 .../storm/topology/base/BaseBasicBolt.java      |    33 -
 .../storm/topology/base/BaseBatchBolt.java      |    24 -
 .../storm/topology/base/BaseComponent.java      |    28 -
 ...BaseOpaquePartitionedTransactionalSpout.java |    25 -
 .../base/BasePartitionedTransactionalSpout.java |    25 -
 .../storm/topology/base/BaseRichBolt.java       |    26 -
 .../storm/topology/base/BaseRichSpout.java      |    46 -
 .../storm/topology/base/BaseStatefulBolt.java   |    64 -
 .../topology/base/BaseStatefulWindowedBolt.java |   168 -
 .../base/BaseTickTupleAwareRichBolt.java        |    60 -
 .../topology/base/BaseTransactionalBolt.java    |    24 -
 .../topology/base/BaseTransactionalSpout.java   |    24 -
 .../storm/topology/base/BaseWindowedBolt.java   |   365 -
 .../apache/storm/transactional/ICommitter.java  |    26 -
 .../ICommitterTransactionalSpout.java           |    31 -
 .../transactional/ITransactionalSpout.java      |    94 -
 .../storm/transactional/TransactionAttempt.java |    61 -
 .../TransactionalSpoutBatchExecutor.java        |    96 -
 .../TransactionalSpoutCoordinator.java          |   212 -
 .../TransactionalTopologyBuilder.java           |   521 -
 .../IOpaquePartitionedTransactionalSpout.java   |    56 -
 .../IPartitionedTransactionalSpout.java         |    69 -
 ...uePartitionedTransactionalSpoutExecutor.java |   160 -
 .../PartitionedTransactionalSpoutExecutor.java  |   142 -
 .../state/RotatingTransactionalState.java       |   149 -
 .../state/TestTransactionalState.java           |    47 -
 .../transactional/state/TransactionalState.java |   173 -
 .../apache/storm/trident/JoinOutFieldsMode.java |    33 -
 .../jvm/org/apache/storm/trident/JoinType.java  |    30 -
 .../jvm/org/apache/storm/trident/Stream.java    |   897 -
 .../org/apache/storm/trident/TridentState.java  |    59 -
 .../apache/storm/trident/TridentTopology.java   |   986 -
 .../trident/drpc/ReturnResultsReducer.java      |   128 -
 .../fluent/ChainedAggregatorDeclarer.java       |   183 -
 .../fluent/ChainedFullAggregatorDeclarer.java   |    32 -
 .../ChainedPartitionAggregatorDeclarer.java     |    32 -
 .../trident/fluent/GlobalAggregationScheme.java |    26 -
 .../storm/trident/fluent/GroupedStream.java     |   174 -
 .../trident/fluent/IAggregatableStream.java     |    31 -
 .../fluent/IChainedAggregatorDeclarer.java      |    24 -
 .../storm/trident/fluent/UniqueIdGen.java       |    38 -
 .../storm/trident/graph/GraphGrouper.java       |   119 -
 .../org/apache/storm/trident/graph/Group.java   |   150 -
 .../storm/trident/operation/Aggregator.java     |    26 -
 .../storm/trident/operation/Assembly.java       |    45 -
 .../storm/trident/operation/BaseAggregator.java |    23 -
 .../storm/trident/operation/BaseFilter.java     |    23 -
 .../storm/trident/operation/BaseFunction.java   |    23 -
 .../trident/operation/BaseMultiReducer.java     |    33 -
 .../storm/trident/operation/BaseOperation.java  |    47 -
 .../trident/operation/CombinerAggregator.java   |    29 -
 .../storm/trident/operation/Consumer.java       |    35 -
 .../operation/DefaultResourceDeclarer.java      |    68 -
 .../storm/trident/operation/EachOperation.java  |    22 -
 .../apache/storm/trident/operation/Filter.java  |    48 -
 .../trident/operation/FlatMapFunction.java      |    37 -
 .../storm/trident/operation/Function.java       |    92 -
 .../trident/operation/GroupedMultiReducer.java  |    31 -
 .../trident/operation/ITridentResource.java     |    32 -
 .../storm/trident/operation/MapFunction.java    |    36 -
 .../storm/trident/operation/MultiReducer.java   |    31 -
 .../storm/trident/operation/Operation.java      |    46 -
 .../OperationAwareFlatMapFunction.java          |    24 -
 .../operation/OperationAwareMapFunction.java    |    24 -
 .../trident/operation/ReducerAggregator.java    |    26 -
 .../trident/operation/TridentCollector.java     |    52 -
 .../operation/TridentMultiReducerContext.java   |    36 -
 .../operation/TridentOperationContext.java      |    65 -
 .../operation/builtin/ComparisonAggregator.java |    91 -
 .../storm/trident/operation/builtin/Count.java  |    41 -
 .../storm/trident/operation/builtin/Debug.java  |    62 -
 .../storm/trident/operation/builtin/Equals.java |    38 -
 .../trident/operation/builtin/FilterNull.java   |    35 -
 .../storm/trident/operation/builtin/FirstN.java |   130 -
 .../storm/trident/operation/builtin/MapGet.java |    38 -
 .../storm/trident/operation/builtin/Max.java    |    37 -
 .../operation/builtin/MaxWithComparator.java    |    51 -
 .../storm/trident/operation/builtin/Min.java    |    36 -
 .../operation/builtin/MinWithComparator.java    |    51 -
 .../storm/trident/operation/builtin/Negate.java |    76 -
 .../trident/operation/builtin/SnapshotGet.java  |    44 -
 .../storm/trident/operation/builtin/Sum.java    |    42 -
 .../operation/builtin/TupleCollectionGet.java   |    46 -
 .../operation/impl/CaptureCollector.java        |    42 -
 .../operation/impl/ChainedAggregatorImpl.java   |   113 -
 .../trident/operation/impl/ChainedResult.java   |    53 -
 .../operation/impl/CombinerAggStateUpdater.java |    56 -
 .../impl/CombinerAggregatorCombineImpl.java     |    61 -
 .../impl/CombinerAggregatorInitImpl.java        |    49 -
 .../operation/impl/ConsumerExecutor.java        |    38 -
 .../trident/operation/impl/FilterExecutor.java  |    53 -
 .../operation/impl/FlatMapFunctionExecutor.java |    64 -
 .../operation/impl/GlobalBatchToPartition.java  |    29 -
 .../trident/operation/impl/GroupCollector.java  |    48 -
 .../operation/impl/GroupedAggregator.java       |    96 -
 .../impl/GroupedMultiReducerExecutor.java       |    95 -
 .../operation/impl/IdentityMultiReducer.java    |    51 -
 .../impl/IndexHashBatchToPartition.java         |    29 -
 .../storm/trident/operation/impl/JoinState.java |    39 -
 .../operation/impl/JoinerMultiReducer.java      |   142 -
 .../operation/impl/MapFunctionExecutor.java     |    62 -
 ...PreservingFieldsOrderJoinerMultiReducer.java |   175 -
 .../operation/impl/ReducerAggStateUpdater.java  |    53 -
 .../operation/impl/ReducerAggregatorImpl.java   |    56 -
 .../storm/trident/operation/impl/Result.java    |    27 -
 .../operation/impl/SingleEmitAggregator.java    |    95 -
 .../trident/operation/impl/TrueFilter.java      |    40 -
 .../storm/trident/partition/GlobalGrouping.java |    42 -
 .../trident/partition/IdentityGrouping.java     |    59 -
 .../trident/partition/IndexHashGrouping.java    |    55 -
 .../storm/trident/planner/BridgeReceiver.java   |    38 -
 .../org/apache/storm/trident/planner/Node.java  |    69 -
 .../storm/trident/planner/NodeStateInfo.java    |    31 -
 .../storm/trident/planner/PartitionNode.java    |    50 -
 .../storm/trident/planner/ProcessorContext.java |    29 -
 .../storm/trident/planner/ProcessorNode.java    |    38 -
 .../apache/storm/trident/planner/SpoutNode.java |    39 -
 .../storm/trident/planner/SubtopologyBolt.java  |   246 -
 .../storm/trident/planner/TridentProcessor.java |    40 -
 .../storm/trident/planner/TupleReceiver.java    |    27 -
 .../planner/processor/AggregateProcessor.java   |    84 -
 .../planner/processor/AppendCollector.java      |    62 -
 .../planner/processor/EachProcessor.java        |    80 -
 .../planner/processor/FreshCollector.java       |    59 -
 .../trident/planner/processor/MapProcessor.java |    87 -
 .../processor/MultiReducerProcessor.java        |    93 -
 .../processor/PartitionPersistProcessor.java    |   107 -
 .../planner/processor/ProjectedProcessor.java   |    73 -
 .../planner/processor/StateQueryProcessor.java  |   107 -
 .../planner/processor/TridentContext.java       |    76 -
 .../storm/trident/spout/BatchSpoutExecutor.java |    92 -
 .../apache/storm/trident/spout/IBatchID.java    |    24 -
 .../apache/storm/trident/spout/IBatchSpout.java |    33 -
 .../trident/spout/ICommitterTridentSpout.java   |    31 -
 .../spout/IOpaquePartitionedTridentSpout.java   |    87 -
 .../trident/spout/IPartitionedTridentSpout.java |    77 -
 .../storm/trident/spout/ISpoutPartition.java    |    25 -
 .../storm/trident/spout/ITridentDataSource.java |    26 -
 .../storm/trident/spout/ITridentSpout.java      |   125 -
 .../OpaquePartitionedTridentSpoutExecutor.java  |   234 -
 .../spout/PartitionedTridentSpoutExecutor.java  |   188 -
 .../trident/spout/RichSpoutBatchExecutor.java   |   204 -
 .../storm/trident/spout/RichSpoutBatchId.java   |    49 -
 .../spout/RichSpoutBatchIdSerializer.java       |    38 -
 .../trident/spout/RichSpoutBatchTriggerer.java  |   182 -
 .../trident/spout/TridentSpoutCoordinator.java  |    94 -
 .../trident/spout/TridentSpoutExecutor.java     |   138 -
 .../storm/trident/state/BaseQueryFunction.java  |    25 -
 .../storm/trident/state/BaseStateUpdater.java   |    25 -
 .../trident/state/CombinerValueUpdater.java     |    45 -
 .../storm/trident/state/ITupleCollection.java   |    26 -
 .../state/JSONNonTransactionalSerializer.java   |    45 -
 .../trident/state/JSONOpaqueSerializer.java     |    52 -
 .../state/JSONTransactionalSerializer.java      |    50 -
 .../apache/storm/trident/state/OpaqueValue.java |    75 -
 .../storm/trident/state/QueryFunction.java      |    28 -
 .../storm/trident/state/ReadOnlyState.java      |    31 -
 .../trident/state/ReducerValueUpdater.java      |    41 -
 .../apache/storm/trident/state/Serializer.java  |    26 -
 .../org/apache/storm/trident/state/State.java   |    39 -
 .../storm/trident/state/StateFactory.java       |    26 -
 .../apache/storm/trident/state/StateSpec.java   |    30 -
 .../apache/storm/trident/state/StateType.java   |    25 -
 .../storm/trident/state/StateUpdater.java       |    33 -
 .../storm/trident/state/TransactionalValue.java |    44 -
 .../storm/trident/state/ValueUpdater.java       |    23 -
 .../trident/state/map/CachedBatchReadsMap.java  |    80 -
 .../storm/trident/state/map/CachedMap.java      |    78 -
 .../storm/trident/state/map/IBackingMap.java    |    26 -
 .../state/map/MapCombinerAggStateUpdater.java   |    84 -
 .../state/map/MapReducerAggStateUpdater.java    |    91 -
 .../storm/trident/state/map/MapState.java       |    26 -
 .../state/map/MicroBatchIBackingMap.java        |    85 -
 .../trident/state/map/NonTransactionalMap.java  |    67 -
 .../storm/trident/state/map/OpaqueMap.java      |   124 -
 .../trident/state/map/ReadOnlyMapState.java     |    26 -
 .../trident/state/map/RemovableMapState.java    |    25 -
 .../trident/state/map/SnapshottableMap.java     |    76 -
 .../trident/state/map/TransactionalMap.java     |   109 -
 .../state/snapshot/ReadOnlySnapshottable.java   |    24 -
 .../trident/state/snapshot/Snapshottable.java   |    27 -
 .../trident/testing/CountAsAggregator.java      |    47 -
 .../storm/trident/testing/FeederBatchSpout.java |   185 -
 .../testing/FeederCommitterBatchSpout.java      |    96 -
 .../storm/trident/testing/FixedBatchSpout.java  |    97 -
 .../apache/storm/trident/testing/IFeeder.java   |    23 -
 .../trident/testing/LRUMemoryMapState.java      |   154 -
 .../storm/trident/testing/MemoryBackingMap.java |    47 -
 .../storm/trident/testing/MemoryMapState.java   |   176 -
 .../org/apache/storm/trident/testing/Split.java |    36 -
 .../storm/trident/testing/StringLength.java     |    32 -
 .../storm/trident/testing/TrueFilter.java       |    30 -
 .../storm/trident/testing/TuplifyArgs.java      |    41 -
 .../storm/trident/topology/BatchInfo.java       |    33 -
 .../trident/topology/ITridentBatchBolt.java     |    32 -
 .../topology/MasterBatchCoordinator.java        |   316 -
 .../trident/topology/TransactionAttempt.java    |    66 -
 .../trident/topology/TridentBoltExecutor.java   |   439 -
 .../topology/TridentTopologyBuilder.java        |   751 -
 .../state/RotatingTransactionalState.java       |   186 -
 .../topology/state/TestTransactionalState.java  |    47 -
 .../topology/state/TransactionalState.java      |   191 -
 .../apache/storm/trident/tuple/ComboList.java   |    92 -
 .../apache/storm/trident/tuple/ConsList.java    |    60 -
 .../storm/trident/tuple/TridentTuple.java       |    34 -
 .../storm/trident/tuple/TridentTupleView.java   |   361 -
 .../storm/trident/tuple/ValuePointer.java       |    60 -
 .../storm/trident/util/ErrorEdgeFactory.java    |    28 -
 .../apache/storm/trident/util/IndexedEdge.java  |    50 -
 .../org/apache/storm/trident/util/LRUMap.java   |    35 -
 .../apache/storm/trident/util/TridentUtils.java |   117 -
 .../windowing/AbstractTridentWindowManager.java |   238 -
 .../windowing/ITridentWindowManager.java        |    59 -
 .../windowing/InMemoryTridentWindowManager.java |    72 -
 .../trident/windowing/InMemoryWindowsStore.java |   200 -
 .../windowing/InMemoryWindowsStoreFactory.java  |    44 -
 .../StoreBasedTridentWindowManager.java         |   217 -
 .../trident/windowing/TridentBatchTuple.java    |    42 -
 .../trident/windowing/WindowKryoSerializer.java |    87 -
 .../windowing/WindowTridentProcessor.java       |   265 -
 .../storm/trident/windowing/WindowsState.java   |    52 -
 .../trident/windowing/WindowsStateFactory.java  |    40 -
 .../trident/windowing/WindowsStateUpdater.java  |    81 -
 .../storm/trident/windowing/WindowsStore.java   |    79 -
 .../trident/windowing/WindowsStoreFactory.java  |    40 -
 .../windowing/config/BaseWindowConfig.java      |    48 -
 .../windowing/config/SlidingCountWindow.java    |    43 -
 .../windowing/config/SlidingDurationWindow.java |    44 -
 .../windowing/config/TumblingCountWindow.java   |    43 -
 .../config/TumblingDurationWindow.java          |    42 -
 .../trident/windowing/config/WindowConfig.java  |    57 -
 .../windowing/strategy/BaseWindowStrategy.java  |    32 -
 .../strategy/SlidingCountWindowStrategy.java    |    59 -
 .../strategy/SlidingDurationWindowStrategy.java |    60 -
 .../strategy/TumblingCountWindowStrategy.java   |    60 -
 .../TumblingDurationWindowStrategy.java         |    60 -
 .../windowing/strategy/WindowStrategy.java      |    45 -
 .../org/apache/storm/tuple/AddressedTuple.java  |    48 -
 .../src/jvm/org/apache/storm/tuple/Fields.java  |   142 -
 .../src/jvm/org/apache/storm/tuple/ITuple.java  |   214 -
 .../jvm/org/apache/storm/tuple/MessageId.java   |    94 -
 .../src/jvm/org/apache/storm/tuple/Tuple.java   |    68 -
 .../jvm/org/apache/storm/tuple/TupleImpl.java   |   356 -
 .../src/jvm/org/apache/storm/tuple/Values.java  |    37 -
 .../src/jvm/org/apache/storm/ui/UIHelpers.java  |     8 +-
 .../jvm/org/apache/storm/utils/BatchHelper.java |    92 -
 .../storm/utils/BufferFileInputStream.java      |    54 -
 .../apache/storm/utils/BufferInputStream.java   |    54 -
 .../apache/storm/utils/CRC32OutputStream.java   |    44 -
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   536 -
 .../jvm/org/apache/storm/utils/DRPCClient.java  |    66 -
 .../utils/DisruptorBackpressureCallback.java    |    27 -
 .../org/apache/storm/utils/DisruptorQueue.java  |   589 -
 .../storm/utils/ExtendedThreadPoolExecutor.java |    67 -
 .../jvm/org/apache/storm/utils/IPredicate.java  |    27 -
 .../storm/utils/IndifferentAccessMap.java       |   177 -
 .../org/apache/storm/utils/InprocMessaging.java |   100 -
 .../storm/utils/KeyedRoundRobinQueue.java       |    68 -
 .../org/apache/storm/utils/ListDelegate.java    |   156 -
 .../jvm/org/apache/storm/utils/LocalState.java  |   277 -
 .../jvm/org/apache/storm/utils/MutableInt.java  |    43 -
 .../jvm/org/apache/storm/utils/MutableLong.java |    43 -
 .../org/apache/storm/utils/MutableObject.java   |    38 -
 .../org/apache/storm/utils/NimbusClient.java    |   139 -
 .../utils/NimbusLeaderNotFoundException.java    |    41 -
 .../storm/utils/RegisteredGlobalState.java      |    62 -
 .../jvm/org/apache/storm/utils/RotatingMap.java |   128 -
 .../org/apache/storm/utils/ServiceRegistry.java |    47 -
 .../storm/utils/ShellBoltMessageQueue.java      |   121 -
 .../org/apache/storm/utils/ShellProcess.java    |   215 -
 .../jvm/org/apache/storm/utils/ShellUtils.java  |   505 -
 .../StormBoundedExponentialBackoffRetry.java    |    79 -
 .../storm/utils/StormCommonInstaller.java       |    43 -
 .../utils/StormConnectionStateConverter.java    |    44 -
 .../apache/storm/utils/ThriftTopologyUtils.java |    70 -
 .../src/jvm/org/apache/storm/utils/Time.java    |   235 -
 .../org/apache/storm/utils/TimeCacheMap.java    |   125 -
 .../apache/storm/utils/TopologySpoutLag.java    |     2 +
 .../org/apache/storm/utils/TransferDrainer.java |   140 -
 .../jvm/org/apache/storm/utils/TupleUtils.java  |    67 -
 .../src/jvm/org/apache/storm/utils/Utils.java   |  2320 -
 .../jvm/org/apache/storm/utils/VersionInfo.java |   131 -
 .../org/apache/storm/utils/VersionedStore.java  |   188 -
 .../jvm/org/apache/storm/utils/WebAppUtils.java |    31 +
 .../storm/utils/WindowedTimeThrottler.java      |    51 -
 .../storm/utils/WorkerBackpressureCallback.java |    26 -
 .../storm/utils/WorkerBackpressureThread.java   |    80 -
 .../org/apache/storm/utils/WritableUtils.java   |   375 -
 .../apache/storm/utils/ZookeeperAuthInfo.java   |    53 -
 .../storm/utils/ZookeeperServerCnxnFactory.java |    84 -
 .../storm/validation/ConfigValidation.java      |   716 -
 .../validation/ConfigValidationAnnotations.java |   218 -
 .../storm/validation/ConfigValidationUtils.java |   175 -
 .../storm/windowing/CountEvictionPolicy.java    |    81 -
 .../storm/windowing/CountTriggerPolicy.java     |    76 -
 .../storm/windowing/DefaultEvictionContext.java |    53 -
 .../jvm/org/apache/storm/windowing/Event.java   |    49 -
 .../org/apache/storm/windowing/EventImpl.java   |    51 -
 .../apache/storm/windowing/EvictionContext.java |    47 -
 .../apache/storm/windowing/EvictionPolicy.java  |    82 -
 .../storm/windowing/TimeEvictionPolicy.java     |    72 -
 .../storm/windowing/TimeTriggerPolicy.java      |   132 -
 .../storm/windowing/TimestampExtractor.java     |    35 -
 .../apache/storm/windowing/TriggerHandler.java  |    31 -
 .../apache/storm/windowing/TriggerPolicy.java   |    49 -
 .../org/apache/storm/windowing/TupleWindow.java |    26 -
 .../apache/storm/windowing/TupleWindowImpl.java |    93 -
 .../apache/storm/windowing/WaterMarkEvent.java  |    38 -
 .../windowing/WaterMarkEventGenerator.java      |   121 -
 .../windowing/WatermarkCountEvictionPolicy.java |    68 -
 .../windowing/WatermarkCountTriggerPolicy.java  |    91 -
 .../windowing/WatermarkTimeEvictionPolicy.java  |    77 -
 .../windowing/WatermarkTimeTriggerPolicy.java   |   126 -
 .../jvm/org/apache/storm/windowing/Window.java  |    55 -
 .../windowing/WindowLifecycleListener.java      |    43 -
 .../apache/storm/windowing/WindowManager.java   |   289 -
 .../storm/zookeeper/LeaderElectorImp.java       |   124 -
 .../apache/storm/zookeeper/ZkEventTypes.java    |    43 -
 .../apache/storm/zookeeper/ZkKeeperStates.java  |    41 -
 .../org/apache/storm/zookeeper/Zookeeper.java   |   545 -
 storm-core/src/py/__init__.py                   |    18 -
 storm-core/src/py/storm/DistributedRPC-remote   |   112 -
 storm-core/src/py/storm/DistributedRPC.py       |   311 -
 .../py/storm/DistributedRPCInvocations-remote   |   133 -
 .../src/py/storm/DistributedRPCInvocations.py   |   886 -
 storm-core/src/py/storm/Nimbus-remote           |   420 -
 storm-core/src/py/storm/Nimbus.py               |  9915 ----
 storm-core/src/py/storm/__init__.py             |    19 -
 storm-core/src/py/storm/constants.py            |    29 -
 storm-core/src/py/storm/ttypes.py               | 11695 ----
 storm-core/src/py_license_header.txt            |    18 -
 storm-core/src/storm.thrift                     |   775 -
 .../org/apache/storm/integration_test.clj       |     6 +-
 .../org/apache/storm/testing4j_test.clj         |     8 +-
 .../test/clj/org/apache/storm/cluster_test.clj  |    14 +-
 .../test/clj/org/apache/storm/drpc_test.clj     |    21 +-
 .../test/clj/org/apache/storm/grouping_test.clj |    28 +-
 .../clj/org/apache/storm/logviewer_test.clj     |     2 +-
 .../apache/storm/messaging/netty_unit_test.clj  |    36 +-
 .../test/clj/org/apache/storm/metrics_test.clj  |    11 +-
 .../test/clj/org/apache/storm/nimbus_test.clj   |    18 +-
 .../apache/storm/security/auth/auth_test.clj    |    17 +-
 .../storm/security/auth/nimbus_auth_test.clj    |    18 +-
 .../clj/org/apache/storm/trident/state_test.clj |     3 +-
 .../jvm/org/apache/storm/MessagingTest.java     |    68 -
 .../storm/PaceMakerStateStorageFactoryTest.java |   155 -
 .../jvm/org/apache/storm/PacemakerTest.java     |   242 -
 .../test/jvm/org/apache/storm/TestCgroups.java  |   129 -
 .../org/apache/storm/TestConfigValidate.java    |   801 -
 .../jvm/org/apache/storm/TickTupleTest.java     |   102 -
 .../apache/storm/blobstore/BlobStoreTest.java   |   469 -
 .../storm/blobstore/BlobSynchronizerTest.java   |   137 -
 .../storm/blobstore/ClientBlobStoreTest.java    |   179 -
 .../jvm/org/apache/storm/bolt/TestJoinBolt.java |   357 -
 .../cluster/StormClusterStateImplTest.java      |   116 -
 .../org/apache/storm/daemon/drpc/DRPCTest.java  |   252 -
 .../daemon/supervisor/BasicContainerTest.java   |   485 -
 .../storm/daemon/supervisor/ContainerTest.java  |   269 -
 .../storm/daemon/supervisor/SlotTest.java       |   498 -
 .../daemon/worker/LogConfigManagerTest.java     |   202 -
 .../apache/storm/daemon/worker/WorkerTest.java  |    39 -
 .../DependencyPropertiesParserTest.java         |    72 -
 .../dependency/DependencyUploaderTest.java      |   297 -
 .../storm/executor/error/ReportErrorTest.java   |    74 -
 .../storm/grouping/PartialKeyGroupingTest.java  |    66 -
 .../storm/grouping/ShuffleGroupingTest.java     |   147 -
 .../storm/localizer/AsyncLocalizerTest.java     |   187 -
 .../LocalizedResourceRetentionSetTest.java      |    85 -
 .../localizer/LocalizedResourceSetTest.java     |    74 -
 .../apache/storm/localizer/LocalizerTest.java   |   672 -
 .../org/apache/storm/localizer/localtest.zip    |   Bin 6378 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.jar    |   Bin 6591 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.tar    |   Bin 24576 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.tar.gz |   Bin 6106 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.tgz    |   Bin 6106 -> 0 bytes
 .../ClusterMetricsConsumerExecutorTest.java     |   133 -
 .../metric/filter/FilterByMetricNameTest.java   |    95 -
 .../metric/internal/CountStatAndMetricTest.java |    86 -
 .../internal/LatencyStatAndMetricTest.java      |    83 -
 .../storm/metric/internal/RateTrackerTest.java  |    94 -
 .../metric/util/DataPointExpanderTest.java      |   109 -
 .../apache/storm/nimbus/LocalNimbusTest.java    |   132 -
 .../resource/TestResourceAwareScheduler.java    |  1366 -
 .../storm/scheduler/resource/TestUser.java      |   111 -
 .../TestUtilsForResourceAwareScheduler.java     |   369 -
 .../eviction/TestDefaultEvictionStrategy.java   |   751 -
 .../TestDefaultResourceAwareStrategy.java       |   331 -
 .../storm/security/auth/AuthUtilsTest.java      |   240 -
 .../storm/security/auth/AuthUtilsTestMock.java  |    82 -
 .../apache/storm/security/auth/AutoSSLTest.java |   136 -
 .../auth/DefaultHttpCredentialsPluginTest.java  |    90 -
 .../storm/security/auth/ReqContextTest.java     |    87 -
 .../security/auth/SaslTransportPluginTest.java  |    49 -
 .../storm/security/auth/ThriftClientTest.java   |    68 -
 .../storm/security/auth/ThriftServerTest.java   |    38 -
 .../authorizer/DRPCSimpleACLAuthorizerTest.java |   163 -
 .../BlowfishTupleSerializerTest.java            |    86 -
 .../GzipBridgeSerializationDelegateTest.java    |    82 -
 ...ipBridgeThriftSerializationDelegateTest.java |    71 -
 .../serialization/SerializationFactoryTest.java |    61 -
 .../storm/serialization/SerializationTest.java  |     7 +-
 .../ThriftBridgeSerializationDelegateTest.java  |    60 -
 .../apache/storm/spout/CheckpointSpoutTest.java |   262 -
 .../storm/state/InMemoryKeyValueStateTest.java  |    99 -
 .../apache/storm/streams/ProcessorBoltTest.java |   174 -
 .../streams/StatefulProcessorBoltTest.java      |   111 -
 .../apache/storm/streams/StreamBuilderTest.java |   268 -
 .../streams/WindowedProcessorBoltTest.java      |   110 -
 .../streams/processors/JoinProcessorTest.java   |   125 -
 .../topology/StatefulBoltExecutorTest.java      |   178 -
 .../StatefulWindowedBoltExecutorTest.java       |   131 -
 .../storm/topology/TopologyBuilderTest.java     |   118 -
 .../topology/WindowedBoltExecutorTest.java      |   230 -
 .../storm/trident/TestTridentTopology.java      |    76 -
 .../storm/trident/TridentWindowingTest.java     |   105 -
 .../jvm/org/apache/storm/tuple/FieldsTest.java  |   126 -
 .../utils/DisruptorQueueBackpressureTest.java   |   110 -
 .../apache/storm/utils/DisruptorQueueTest.java  |   187 -
 .../apache/storm/utils/MockTupleHelpers.java    |    40 -
 .../storm/utils/ShellBoltMessageQueueTest.java  |    85 -
 ...StormBoundedExponentialBackoffRetryTest.java |   101 -
 .../storm/utils/ThriftTopologyUtilsTest.java    |   106 -
 .../utils/ThrowableNestedCauseMatcher.java      |    44 -
 .../jvm/org/apache/storm/utils/TimeTest.java    |    95 -
 .../jvm/org/apache/storm/utils/UtilsTest.java   |   219 -
 .../utils/WorkerBackpressureThreadTest.java     |    50 -
 .../utils/ZookeeperServerCnxnFactoryTest.java   |    39 -
 .../staticmocking/ConfigUtilsInstaller.java     |    13 +-
 .../staticmocking/ReflectionUtilsInstaller.java |    39 +
 .../ServerConfigUtilsInstaller.java             |    41 +
 .../utils/staticmocking/UtilsInstaller.java     |    13 +-
 .../windowing/WaterMarkEventGeneratorTest.java  |   118 -
 .../storm/windowing/WindowManagerTest.java      |   570 -
 .../final-package/src/main/assembly/binary.xml  |     9 +-
 storm-dist/binary/pom.xml                       |     3 +-
 storm-dist/binary/storm-client-bin/pom.xml      |    65 +
 .../src/main/assembly/storm-client.xml          |    33 +
 storm-dist/binary/storm-drpc-server-bin/pom.xml |    64 -
 .../src/main/assembly/storm-drpc-server.xml     |    32 -
 storm-dist/binary/storm-webapp-bin/pom.xml      |    64 +
 .../src/main/assembly/storm-webapp.xml          |    32 +
 storm-drpc-server/pom.xml                       |   162 -
 .../apache/storm/daemon/drpc/DRPCServer.java    |   180 -
 .../webapp/AuthorizationExceptionMapper.java    |    39 -
 .../daemon/drpc/webapp/DRPCApplication.java     |    47 -
 .../daemon/drpc/webapp/DRPCExceptionMapper.java |    61 -
 .../storm/daemon/drpc/webapp/DRPCResource.java  |    63 -
 .../daemon/drpc/webapp/ReqContextFilter.java    |    69 -
 .../storm/daemon/drpc/DRPCServerTest.java       |   214 -
 storm-server/pom.xml                            |   157 +
 .../java/org/apache/storm/DaemonConfig.java     |   916 +
 .../ILocalClusterTrackedTopologyAware.java      |    54 +
 .../java/org/apache/storm/LocalCluster.java     |   834 +
 .../main/java/org/apache/storm/LocalDRPC.java   |    90 +
 .../java/org/apache/storm/ProcessSimulator.java |    97 +
 .../src/main/java/org/apache/storm/Testing.java |   716 +
 .../storm/blobstore/BlobKeySequenceInfo.java    |    40 +
 .../apache/storm/blobstore/BlobStoreUtils.java  |   270 +
 .../storm/blobstore/BlobSynchronizer.java       |   131 +
 .../storm/blobstore/FileBlobStoreImpl.java      |   269 +
 .../storm/blobstore/KeySequenceNumber.java      |   237 +
 .../storm/blobstore/LocalFsBlobStore.java       |   351 +
 .../storm/blobstore/LocalFsBlobStoreFile.java   |   159 +
 .../container/ResourceIsolationInterface.java   |    76 +
 .../storm/container/cgroup/CgroupManager.java   |   224 +
 .../daemon/drpc/BlockingOutstandingRequest.java |    67 +
 .../java/org/apache/storm/daemon/drpc/DRPC.java |   215 +
 .../apache/storm/daemon/drpc/DRPCThrift.java    |    58 +
 .../storm/daemon/drpc/OutstandingRequest.java   |    58 +
 .../storm/daemon/drpc/RequestFactory.java       |    24 +
 .../storm/daemon/metrics/MetricsUtils.java      |   109 +
 .../reporters/ConsolePreparableReporter.java    |    76 +
 .../reporters/CsvPreparableReporter.java        |    80 +
 .../reporters/JmxPreparableReporter.java        |    71 +
 .../metrics/reporters/PreparableReporter.java   |    32 +
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |  3807 ++
 .../storm/daemon/nimbus/TopologyActions.java    |    31 +
 .../storm/daemon/nimbus/TopologyResources.java  |    63 +
 .../daemon/nimbus/TopologyStateTransition.java  |    27 +
 .../storm/daemon/supervisor/BasicContainer.java |   723 +
 .../supervisor/BasicContainerLauncher.java      |    62 +
 .../storm/daemon/supervisor/Container.java      |   567 +
 .../daemon/supervisor/ContainerLauncher.java    |   106 +
 .../supervisor/ContainerRecoveryException.java  |    29 +
 .../DefaultUncaughtExceptionHandler.java        |    31 +
 .../supervisor/EventManagerPushCallback.java    |    37 +
 .../storm/daemon/supervisor/Killable.java       |    50 +
 .../storm/daemon/supervisor/LocalContainer.java |    94 +
 .../supervisor/LocalContainerLauncher.java      |    60 +
 .../daemon/supervisor/ReadClusterState.java     |   358 +
 .../daemon/supervisor/RunAsUserContainer.java   |   101 +
 .../supervisor/RunAsUserContainerLauncher.java  |    60 +
 .../apache/storm/daemon/supervisor/Slot.java    |   787 +
 .../daemon/supervisor/StandaloneSupervisor.java |    91 +
 .../storm/daemon/supervisor/Supervisor.java     |   362 +
 .../daemon/supervisor/SupervisorUtils.java      |   191 +
 .../apache/storm/daemon/supervisor/UniFunc.java |    22 +
 .../supervisor/timer/SupervisorHealthCheck.java |    41 +
 .../supervisor/timer/SupervisorHeartbeat.java   |    88 +
 .../daemon/supervisor/timer/UpdateBlobs.java    |   105 +
 .../org/apache/storm/event/EventManager.java    |    24 +
 .../org/apache/storm/event/EventManagerImp.java |   100 +
 .../apache/storm/healthcheck/HealthChecker.java |   122 +
 .../apache/storm/localizer/AsyncLocalizer.java  |   437 +
 .../org/apache/storm/localizer/ILocalizer.java  |    70 +
 .../localizer/LocalDownloadedResource.java      |   146 +
 .../apache/storm/localizer/LocalResource.java   |    44 +
 .../storm/localizer/LocalizedResource.java      |   130 +
 .../LocalizedResourceRetentionSet.java          |   140 +
 .../storm/localizer/LocalizedResourceSet.java   |   101 +
 .../org/apache/storm/localizer/Localizer.java   |   704 +
 .../storm/logging/ThriftAccessLogger.java       |    56 +
 .../logging/filters/AccessLoggingFilter.java    |    52 +
 .../metric/ClusterMetricsConsumerExecutor.java  |    89 +
 .../metric/LoggingClusterMetricsConsumer.java   |    86 +
 .../storm/metric/StormMetricsRegistry.java      |    88 +
 .../org/apache/storm/metric/api/DataPoint.java  |    43 +
 .../metric/api/IClusterMetricsConsumer.java     |    63 +
 .../storm/nimbus/DefaultTopologyValidator.java  |    31 +
 .../org/apache/storm/nimbus/ILeaderElector.java |    73 +
 .../nimbus/ITopologyActionNotifierPlugin.java   |    43 +
 .../apache/storm/nimbus/ITopologyValidator.java |    28 +
 .../storm/pacemaker/IServerMessageHandler.java  |    25 +
 .../org/apache/storm/pacemaker/Pacemaker.java   |   218 +
 .../apache/storm/pacemaker/PacemakerServer.java |   169 +
 .../pacemaker/codec/ThriftNettyServerCodec.java |   100 +
 .../storm/scheduler/DefaultScheduler.java       |   111 +
 .../apache/storm/scheduler/EvenScheduler.java   |   172 +
 .../storm/scheduler/IsolationScheduler.java     |   418 +
 .../scheduler/multitenant/DefaultPool.java      |   219 +
 .../storm/scheduler/multitenant/FreePool.java   |   125 +
 .../scheduler/multitenant/IsolatedPool.java     |   371 +
 .../multitenant/MultitenantScheduler.java       |    99 +
 .../storm/scheduler/multitenant/Node.java       |   340 +
 .../storm/scheduler/multitenant/NodePool.java   |   296 +
 .../resource/ResourceAwareScheduler.java        |   437 +
 .../eviction/DefaultEvictionStrategy.java       |   126 +
 .../strategies/eviction/IEvictionStrategy.java  |    42 +
 .../DefaultSchedulingPriorityStrategy.java      |    80 +
 .../priority/ISchedulingPriorityStrategy.java   |    36 +
 .../storm/testing/CompleteTopologyParam.java    |   103 +
 .../storm/testing/InProcessZookeeper.java       |    59 +
 .../java/org/apache/storm/testing/TestJob.java  |    43 +
 .../apache/storm/testing/TrackedTopology.java   |   135 +
 .../storm/topology/ConfigurableTopology.java    |   184 +
 .../apache/storm/utils/BufferInputStream.java   |    54 +
 .../apache/storm/utils/ServerConfigUtils.java   |   199 +
 .../org/apache/storm/utils/ServerUtils.java     |   772 +
 .../storm/utils/StormCommonInstaller.java       |    43 +
 .../storm/zookeeper/LeaderElectorImp.java       |   123 +
 .../org/apache/storm/zookeeper/Zookeeper.java   |   261 +
 .../java/org/apache/storm/MessagingTest.java    |    68 +
 .../java/org/apache/storm/PacemakerTest.java    |   242 +
 .../test/java/org/apache/storm/TestCgroups.java |   129 +
 .../apache/storm/TestDaemonConfigValidate.java  |   110 +
 .../java/org/apache/storm/TickTupleTest.java    |   102 +
 .../apache/storm/blobstore/BlobStoreTest.java   |   469 +
 .../storm/blobstore/BlobSynchronizerTest.java   |   137 +
 .../org/apache/storm/daemon/drpc/DRPCTest.java  |   253 +
 .../daemon/supervisor/BasicContainerTest.java   |   486 +
 .../storm/daemon/supervisor/ContainerTest.java  |   270 +
 .../storm/daemon/supervisor/SlotTest.java       |   498 +
 .../storm/localizer/AsyncLocalizerTest.java     |   187 +
 .../LocalizedResourceRetentionSetTest.java      |    85 +
 .../localizer/LocalizedResourceSetTest.java     |    74 +
 .../apache/storm/localizer/LocalizerTest.java   |   679 +
 .../ClusterMetricsConsumerExecutorTest.java     |   133 +
 .../apache/storm/nimbus/LocalNimbusTest.java    |   132 +
 .../resource/TestResourceAwareScheduler.java    |  1367 +
 .../storm/scheduler/resource/TestUser.java      |   111 +
 .../TestUtilsForResourceAwareScheduler.java     |   369 +
 .../eviction/TestDefaultEvictionStrategy.java   |   752 +
 .../TestDefaultResourceAwareStrategy.java       |   332 +
 .../src/test/resources/localizer/localtest.zip  |   Bin 0 -> 6378 bytes
 .../localizer/localtestwithsymlink.jar          |   Bin 0 -> 6591 bytes
 .../localizer/localtestwithsymlink.tar          |   Bin 0 -> 24576 bytes
 .../localizer/localtestwithsymlink.tar.gz       |   Bin 0 -> 6106 bytes
 .../localizer/localtestwithsymlink.tgz          |   Bin 0 -> 6106 bytes
 storm-submit-tools/pom.xml                      |     4 +-
 storm-webapp/pom.xml                            |   163 +
 .../apache/storm/daemon/drpc/DRPCServer.java    |   182 +
 .../webapp/AuthorizationExceptionMapper.java    |    39 +
 .../daemon/drpc/webapp/DRPCApplication.java     |    47 +
 .../daemon/drpc/webapp/DRPCExceptionMapper.java |    61 +
 .../storm/daemon/drpc/webapp/DRPCResource.java  |    63 +
 .../daemon/drpc/webapp/ReqContextFilter.java    |    69 +
 .../storm/daemon/drpc/DRPCServerTest.java       |   214 +
 2437 files changed, 266439 insertions(+), 265276 deletions(-)
----------------------------------------------------------------------



[06/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologySummary.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologySummary.java b/storm-client/src/jvm/org/apache/storm/generated/TopologySummary.java
new file mode 100644
index 0000000..8b7cd75
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologySummary.java
@@ -0,0 +1,1901 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class TopologySummary implements org.apache.thrift.TBase<TopologySummary, TopologySummary._Fields>, java.io.Serializable, Cloneable, Comparable<TopologySummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologySummary");
+
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
+  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)514);
+  private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)515);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologySummaryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologySummaryTupleSchemeFactory());
+  }
+
+  private String id; // required
+  private String name; // required
+  private int num_tasks; // required
+  private int num_executors; // required
+  private int num_workers; // required
+  private int uptime_secs; // required
+  private String status; // required
+  private String sched_status; // optional
+  private String owner; // optional
+  private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ID((short)1, "id"),
+    NAME((short)2, "name"),
+    NUM_TASKS((short)3, "num_tasks"),
+    NUM_EXECUTORS((short)4, "num_executors"),
+    NUM_WORKERS((short)5, "num_workers"),
+    UPTIME_SECS((short)6, "uptime_secs"),
+    STATUS((short)7, "status"),
+    SCHED_STATUS((short)513, "sched_status"),
+    OWNER((short)514, "owner"),
+    REPLICATION_COUNT((short)515, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // NAME
+          return NAME;
+        case 3: // NUM_TASKS
+          return NUM_TASKS;
+        case 4: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 5: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 6: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 7: // STATUS
+          return STATUS;
+        case 513: // SCHED_STATUS
+          return SCHED_STATUS;
+        case 514: // OWNER
+          return OWNER;
+        case 515: // REPLICATION_COUNT
+          return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_TASKS_ISSET_ID = 0;
+  private static final int __NUM_EXECUTORS_ISSET_ID = 1;
+  private static final int __NUM_WORKERS_ISSET_ID = 2;
+  private static final int __UPTIME_SECS_ISSET_ID = 3;
+  private static final int __REPLICATION_COUNT_ISSET_ID = 4;
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __REQUESTED_CPU_ISSET_ID = 7;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 8;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 9;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 10;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap);
+  }
+
+  public TopologySummary() {
+  }
+
+  public TopologySummary(
+    String id,
+    String name,
+    int num_tasks,
+    int num_executors,
+    int num_workers,
+    int uptime_secs,
+    String status)
+  {
+    this();
+    this.id = id;
+    this.name = name;
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+    this.status = status;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologySummary(TopologySummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.num_tasks = other.num_tasks;
+    this.num_executors = other.num_executors;
+    this.num_workers = other.num_workers;
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_status()) {
+      this.status = other.status;
+    }
+    if (other.is_set_sched_status()) {
+      this.sched_status = other.sched_status;
+    }
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
+    this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
+  }
+
+  public TopologySummary deepCopy() {
+    return new TopologySummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.id = null;
+    this.name = null;
+    set_num_tasks_isSet(false);
+    this.num_tasks = 0;
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    set_num_workers_isSet(false);
+    this.num_workers = 0;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    this.status = null;
+    this.sched_status = null;
+    this.owner = null;
+    set_replication_count_isSet(false);
+    this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value);
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_status() {
+    return this.status;
+  }
+
+  public void set_status(String status) {
+    this.status = status;
+  }
+
+  public void unset_status() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_status() {
+    return this.status != null;
+  }
+
+  public void set_status_isSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public String get_sched_status() {
+    return this.sched_status;
+  }
+
+  public void set_sched_status(String sched_status) {
+    this.sched_status = sched_status;
+  }
+
+  public void unset_sched_status() {
+    this.sched_status = null;
+  }
+
+  /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sched_status() {
+    return this.sched_status != null;
+  }
+
+  public void set_sched_status_isSet(boolean value) {
+    if (!value) {
+      this.sched_status = null;
+    }
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = null;
+  }
+
+  /** Returns true if field owner is set (has been assigned a value) and false otherwise */
+  public boolean is_set_owner() {
+    return this.owner != null;
+  }
+
+  public void set_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
+  public int get_replication_count() {
+    return this.replication_count;
+  }
+
+  public void set_replication_count(int replication_count) {
+    this.replication_count = replication_count;
+    set_replication_count_isSet(true);
+  }
+
+  public void unset_replication_count() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_count() {
+    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  public void set_replication_count_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
+  }
+
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unset_status();
+      } else {
+        set_status((String)value);
+      }
+      break;
+
+    case SCHED_STATUS:
+      if (value == null) {
+        unset_sched_status();
+      } else {
+        set_sched_status((String)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
+    case REPLICATION_COUNT:
+      if (value == null) {
+        unset_replication_count();
+      } else {
+        set_replication_count((Integer)value);
+      }
+      break;
+
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case NAME:
+      return get_name();
+
+    case NUM_TASKS:
+      return get_num_tasks();
+
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case NUM_WORKERS:
+      return get_num_workers();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case STATUS:
+      return get_status();
+
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case OWNER:
+      return get_owner();
+
+    case REPLICATION_COUNT:
+      return get_replication_count();
+
+    case REQUESTED_MEMONHEAP:
+      return get_requested_memonheap();
+
+    case REQUESTED_MEMOFFHEAP:
+      return get_requested_memoffheap();
+
+    case REQUESTED_CPU:
+      return get_requested_cpu();
+
+    case ASSIGNED_MEMONHEAP:
+      return get_assigned_memonheap();
+
+    case ASSIGNED_MEMOFFHEAP:
+      return get_assigned_memoffheap();
+
+    case ASSIGNED_CPU:
+      return get_assigned_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case NAME:
+      return is_set_name();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case STATUS:
+      return is_set_status();
+    case SCHED_STATUS:
+      return is_set_sched_status();
+    case OWNER:
+      return is_set_owner();
+    case REPLICATION_COUNT:
+      return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologySummary)
+      return this.equals((TopologySummary)that);
+    return false;
+  }
+
+  public boolean equals(TopologySummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_num_tasks = true;
+    boolean that_present_num_tasks = true;
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_num_executors = true;
+    boolean that_present_num_executors = true;
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_num_workers = true;
+    boolean that_present_num_workers = true;
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_status = true && this.is_set_status();
+    boolean that_present_status = true && that.is_set_status();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    boolean this_present_sched_status = true && this.is_set_sched_status();
+    boolean that_present_sched_status = true && that.is_set_sched_status();
+    if (this_present_sched_status || that_present_sched_status) {
+      if (!(this_present_sched_status && that_present_sched_status))
+        return false;
+      if (!this.sched_status.equals(that.sched_status))
+        return false;
+    }
+
+    boolean this_present_owner = true && this.is_set_owner();
+    boolean that_present_owner = true && that.is_set_owner();
+    if (this_present_owner || that_present_owner) {
+      if (!(this_present_owner && that_present_owner))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
+    boolean this_present_replication_count = true && this.is_set_replication_count();
+    boolean that_present_replication_count = true && that.is_set_replication_count();
+    if (this_present_replication_count || that_present_replication_count) {
+      if (!(this_present_replication_count && that_present_replication_count))
+        return false;
+      if (this.replication_count != that.replication_count)
+        return false;
+    }
+
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_id = true && (is_set_id());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_num_tasks = true;
+    list.add(present_num_tasks);
+    if (present_num_tasks)
+      list.add(num_tasks);
+
+    boolean present_num_executors = true;
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_num_workers = true;
+    list.add(present_num_workers);
+    if (present_num_workers)
+      list.add(num_workers);
+
+    boolean present_uptime_secs = true;
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_status = true && (is_set_status());
+    list.add(present_status);
+    if (present_status)
+      list.add(status);
+
+    boolean present_sched_status = true && (is_set_sched_status());
+    list.add(present_sched_status);
+    if (present_sched_status)
+      list.add(sched_status);
+
+    boolean present_owner = true && (is_set_owner());
+    list.add(present_owner);
+    if (present_owner)
+      list.add(owner);
+
+    boolean present_replication_count = true && (is_set_replication_count());
+    list.add(present_replication_count);
+    if (present_replication_count)
+      list.add(replication_count);
+
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologySummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(other.is_set_sched_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sched_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, other.sched_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(other.is_set_replication_count());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_count()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, other.replication_count);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologySummary(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_tasks:");
+    sb.append(this.num_tasks);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_executors:");
+    sb.append(this.num_executors);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_workers:");
+    sb.append(this.num_workers);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("status:");
+    if (this.status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.status);
+    }
+    first = false;
+    if (is_set_sched_status()) {
+      if (!first) sb.append(", ");
+      sb.append("sched_status:");
+      if (this.sched_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sched_status);
+      }
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
+    if (is_set_replication_count()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_count:");
+      sb.append(this.replication_count);
+      first = false;
+    }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_tasks()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_tasks' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_executors()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_executors' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_workers()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_status()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologySummaryStandardSchemeFactory implements SchemeFactory {
+    public TopologySummaryStandardScheme getScheme() {
+      return new TopologySummaryStandardScheme();
+    }
+  }
+
+  private static class TopologySummaryStandardScheme extends StandardScheme<TopologySummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologySummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.id = iprot.readString();
+              struct.set_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_tasks = iprot.readI32();
+              struct.set_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // NUM_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_workers = iprot.readI32();
+              struct.set_num_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.status = iprot.readString();
+              struct.set_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 513: // SCHED_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.sched_status = iprot.readString();
+              struct.set_sched_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 514: // OWNER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.owner = iprot.readString();
+              struct.set_owner_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 515: // REPLICATION_COUNT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.replication_count = iprot.readI32();
+              struct.set_replication_count_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologySummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(struct.id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+      oprot.writeI32(struct.num_tasks);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+      oprot.writeI32(struct.num_executors);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+      oprot.writeI32(struct.num_workers);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeFieldEnd();
+      if (struct.status != null) {
+        oprot.writeFieldBegin(STATUS_FIELD_DESC);
+        oprot.writeString(struct.status);
+        oprot.writeFieldEnd();
+      }
+      if (struct.sched_status != null) {
+        if (struct.is_set_sched_status()) {
+          oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
+          oprot.writeString(struct.sched_status);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.owner != null) {
+        if (struct.is_set_owner()) {
+          oprot.writeFieldBegin(OWNER_FIELD_DESC);
+          oprot.writeString(struct.owner);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_replication_count()) {
+        oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC);
+        oprot.writeI32(struct.replication_count);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TopologySummaryTupleSchemeFactory implements SchemeFactory {
+    public TopologySummaryTupleScheme getScheme() {
+      return new TopologySummaryTupleScheme();
+    }
+  }
+
+  private static class TopologySummaryTupleScheme extends TupleScheme<TopologySummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TopologySummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.id);
+      oprot.writeString(struct.name);
+      oprot.writeI32(struct.num_tasks);
+      oprot.writeI32(struct.num_executors);
+      oprot.writeI32(struct.num_workers);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeString(struct.status);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_sched_status()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_owner()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_replication_count()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
+      if (struct.is_set_sched_status()) {
+        oprot.writeString(struct.sched_status);
+      }
+      if (struct.is_set_owner()) {
+        oprot.writeString(struct.owner);
+      }
+      if (struct.is_set_replication_count()) {
+        oprot.writeI32(struct.replication_count);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TopologySummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.id = iprot.readString();
+      struct.set_id_isSet(true);
+      struct.name = iprot.readString();
+      struct.set_name_isSet(true);
+      struct.num_tasks = iprot.readI32();
+      struct.set_num_tasks_isSet(true);
+      struct.num_executors = iprot.readI32();
+      struct.set_num_executors_isSet(true);
+      struct.num_workers = iprot.readI32();
+      struct.set_num_workers_isSet(true);
+      struct.uptime_secs = iprot.readI32();
+      struct.set_uptime_secs_isSet(true);
+      struct.status = iprot.readString();
+      struct.set_status_isSet(true);
+      BitSet incoming = iprot.readBitSet(9);
+      if (incoming.get(0)) {
+        struct.sched_status = iprot.readString();
+        struct.set_sched_status_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.owner = iprot.readString();
+        struct.set_owner_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.replication_count = iprot.readI32();
+        struct.set_replication_count_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
+    }
+  }
+
+}
+


[17/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ProfileRequest.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ProfileRequest.java b/storm-client/src/jvm/org/apache/storm/generated/ProfileRequest.java
new file mode 100644
index 0000000..1189175
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ProfileRequest.java
@@ -0,0 +1,631 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ProfileRequest implements org.apache.thrift.TBase<ProfileRequest, ProfileRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ProfileRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ProfileRequest");
+
+  private static final org.apache.thrift.protocol.TField NODE_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("nodeInfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField ACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("action", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField TIME_STAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("time_stamp", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ProfileRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ProfileRequestTupleSchemeFactory());
+  }
+
+  private NodeInfo nodeInfo; // required
+  private ProfileAction action; // required
+  private long time_stamp; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NODE_INFO((short)1, "nodeInfo"),
+    /**
+     * 
+     * @see ProfileAction
+     */
+    ACTION((short)2, "action"),
+    TIME_STAMP((short)3, "time_stamp");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NODE_INFO
+          return NODE_INFO;
+        case 2: // ACTION
+          return ACTION;
+        case 3: // TIME_STAMP
+          return TIME_STAMP;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_STAMP_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.TIME_STAMP};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NODE_INFO, new org.apache.thrift.meta_data.FieldMetaData("nodeInfo", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NodeInfo.class)));
+    tmpMap.put(_Fields.ACTION, new org.apache.thrift.meta_data.FieldMetaData("action", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ProfileAction.class)));
+    tmpMap.put(_Fields.TIME_STAMP, new org.apache.thrift.meta_data.FieldMetaData("time_stamp", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ProfileRequest.class, metaDataMap);
+  }
+
+  public ProfileRequest() {
+  }
+
+  public ProfileRequest(
+    NodeInfo nodeInfo,
+    ProfileAction action)
+  {
+    this();
+    this.nodeInfo = nodeInfo;
+    this.action = action;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ProfileRequest(ProfileRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_nodeInfo()) {
+      this.nodeInfo = new NodeInfo(other.nodeInfo);
+    }
+    if (other.is_set_action()) {
+      this.action = other.action;
+    }
+    this.time_stamp = other.time_stamp;
+  }
+
+  public ProfileRequest deepCopy() {
+    return new ProfileRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.nodeInfo = null;
+    this.action = null;
+    set_time_stamp_isSet(false);
+    this.time_stamp = 0;
+  }
+
+  public NodeInfo get_nodeInfo() {
+    return this.nodeInfo;
+  }
+
+  public void set_nodeInfo(NodeInfo nodeInfo) {
+    this.nodeInfo = nodeInfo;
+  }
+
+  public void unset_nodeInfo() {
+    this.nodeInfo = null;
+  }
+
+  /** Returns true if field nodeInfo is set (has been assigned a value) and false otherwise */
+  public boolean is_set_nodeInfo() {
+    return this.nodeInfo != null;
+  }
+
+  public void set_nodeInfo_isSet(boolean value) {
+    if (!value) {
+      this.nodeInfo = null;
+    }
+  }
+
+  /**
+   * 
+   * @see ProfileAction
+   */
+  public ProfileAction get_action() {
+    return this.action;
+  }
+
+  /**
+   * 
+   * @see ProfileAction
+   */
+  public void set_action(ProfileAction action) {
+    this.action = action;
+  }
+
+  public void unset_action() {
+    this.action = null;
+  }
+
+  /** Returns true if field action is set (has been assigned a value) and false otherwise */
+  public boolean is_set_action() {
+    return this.action != null;
+  }
+
+  public void set_action_isSet(boolean value) {
+    if (!value) {
+      this.action = null;
+    }
+  }
+
+  public long get_time_stamp() {
+    return this.time_stamp;
+  }
+
+  public void set_time_stamp(long time_stamp) {
+    this.time_stamp = time_stamp;
+    set_time_stamp_isSet(true);
+  }
+
+  public void unset_time_stamp() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  /** Returns true if field time_stamp is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_stamp() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  public void set_time_stamp_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_STAMP_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NODE_INFO:
+      if (value == null) {
+        unset_nodeInfo();
+      } else {
+        set_nodeInfo((NodeInfo)value);
+      }
+      break;
+
+    case ACTION:
+      if (value == null) {
+        unset_action();
+      } else {
+        set_action((ProfileAction)value);
+      }
+      break;
+
+    case TIME_STAMP:
+      if (value == null) {
+        unset_time_stamp();
+      } else {
+        set_time_stamp((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NODE_INFO:
+      return get_nodeInfo();
+
+    case ACTION:
+      return get_action();
+
+    case TIME_STAMP:
+      return get_time_stamp();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NODE_INFO:
+      return is_set_nodeInfo();
+    case ACTION:
+      return is_set_action();
+    case TIME_STAMP:
+      return is_set_time_stamp();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ProfileRequest)
+      return this.equals((ProfileRequest)that);
+    return false;
+  }
+
+  public boolean equals(ProfileRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_nodeInfo = true && this.is_set_nodeInfo();
+    boolean that_present_nodeInfo = true && that.is_set_nodeInfo();
+    if (this_present_nodeInfo || that_present_nodeInfo) {
+      if (!(this_present_nodeInfo && that_present_nodeInfo))
+        return false;
+      if (!this.nodeInfo.equals(that.nodeInfo))
+        return false;
+    }
+
+    boolean this_present_action = true && this.is_set_action();
+    boolean that_present_action = true && that.is_set_action();
+    if (this_present_action || that_present_action) {
+      if (!(this_present_action && that_present_action))
+        return false;
+      if (!this.action.equals(that.action))
+        return false;
+    }
+
+    boolean this_present_time_stamp = true && this.is_set_time_stamp();
+    boolean that_present_time_stamp = true && that.is_set_time_stamp();
+    if (this_present_time_stamp || that_present_time_stamp) {
+      if (!(this_present_time_stamp && that_present_time_stamp))
+        return false;
+      if (this.time_stamp != that.time_stamp)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_nodeInfo = true && (is_set_nodeInfo());
+    list.add(present_nodeInfo);
+    if (present_nodeInfo)
+      list.add(nodeInfo);
+
+    boolean present_action = true && (is_set_action());
+    list.add(present_action);
+    if (present_action)
+      list.add(action.getValue());
+
+    boolean present_time_stamp = true && (is_set_time_stamp());
+    list.add(present_time_stamp);
+    if (present_time_stamp)
+      list.add(time_stamp);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ProfileRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_nodeInfo()).compareTo(other.is_set_nodeInfo());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_nodeInfo()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nodeInfo, other.nodeInfo);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_action()).compareTo(other.is_set_action());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_action()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.action, other.action);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_stamp()).compareTo(other.is_set_time_stamp());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_stamp()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_stamp, other.time_stamp);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ProfileRequest(");
+    boolean first = true;
+
+    sb.append("nodeInfo:");
+    if (this.nodeInfo == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.nodeInfo);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("action:");
+    if (this.action == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.action);
+    }
+    first = false;
+    if (is_set_time_stamp()) {
+      if (!first) sb.append(", ");
+      sb.append("time_stamp:");
+      sb.append(this.time_stamp);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_nodeInfo()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'nodeInfo' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_action()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'action' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (nodeInfo != null) {
+      nodeInfo.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ProfileRequestStandardSchemeFactory implements SchemeFactory {
+    public ProfileRequestStandardScheme getScheme() {
+      return new ProfileRequestStandardScheme();
+    }
+  }
+
+  private static class ProfileRequestStandardScheme extends StandardScheme<ProfileRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ProfileRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NODE_INFO
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.nodeInfo = new NodeInfo();
+              struct.nodeInfo.read(iprot);
+              struct.set_nodeInfo_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // ACTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.action = org.apache.storm.generated.ProfileAction.findByValue(iprot.readI32());
+              struct.set_action_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TIME_STAMP
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.time_stamp = iprot.readI64();
+              struct.set_time_stamp_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ProfileRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.nodeInfo != null) {
+        oprot.writeFieldBegin(NODE_INFO_FIELD_DESC);
+        struct.nodeInfo.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.action != null) {
+        oprot.writeFieldBegin(ACTION_FIELD_DESC);
+        oprot.writeI32(struct.action.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_time_stamp()) {
+        oprot.writeFieldBegin(TIME_STAMP_FIELD_DESC);
+        oprot.writeI64(struct.time_stamp);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ProfileRequestTupleSchemeFactory implements SchemeFactory {
+    public ProfileRequestTupleScheme getScheme() {
+      return new ProfileRequestTupleScheme();
+    }
+  }
+
+  private static class ProfileRequestTupleScheme extends TupleScheme<ProfileRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ProfileRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.nodeInfo.write(oprot);
+      oprot.writeI32(struct.action.getValue());
+      BitSet optionals = new BitSet();
+      if (struct.is_set_time_stamp()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_time_stamp()) {
+        oprot.writeI64(struct.time_stamp);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ProfileRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.nodeInfo = new NodeInfo();
+      struct.nodeInfo.read(iprot);
+      struct.set_nodeInfo_isSet(true);
+      struct.action = org.apache.storm.generated.ProfileAction.findByValue(iprot.readI32());
+      struct.set_action_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.time_stamp = iprot.readI64();
+        struct.set_time_stamp_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ReadableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ReadableBlobMeta.java b/storm-client/src/jvm/org/apache/storm/generated/ReadableBlobMeta.java
new file mode 100644
index 0000000..0cbd3b8
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ReadableBlobMeta.java
@@ -0,0 +1,510 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ReadableBlobMeta implements org.apache.thrift.TBase<ReadableBlobMeta, ReadableBlobMeta._Fields>, java.io.Serializable, Cloneable, Comparable<ReadableBlobMeta> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ReadableBlobMeta");
+
+  private static final org.apache.thrift.protocol.TField SETTABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("settable", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.I64, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ReadableBlobMetaStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ReadableBlobMetaTupleSchemeFactory());
+  }
+
+  private SettableBlobMeta settable; // required
+  private long version; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SETTABLE((short)1, "settable"),
+    VERSION((short)2, "version");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SETTABLE
+          return SETTABLE;
+        case 2: // VERSION
+          return VERSION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __VERSION_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SETTABLE, new org.apache.thrift.meta_data.FieldMetaData("settable", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SettableBlobMeta.class)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ReadableBlobMeta.class, metaDataMap);
+  }
+
+  public ReadableBlobMeta() {
+  }
+
+  public ReadableBlobMeta(
+    SettableBlobMeta settable,
+    long version)
+  {
+    this();
+    this.settable = settable;
+    this.version = version;
+    set_version_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ReadableBlobMeta(ReadableBlobMeta other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_settable()) {
+      this.settable = new SettableBlobMeta(other.settable);
+    }
+    this.version = other.version;
+  }
+
+  public ReadableBlobMeta deepCopy() {
+    return new ReadableBlobMeta(this);
+  }
+
+  @Override
+  public void clear() {
+    this.settable = null;
+    set_version_isSet(false);
+    this.version = 0;
+  }
+
+  public SettableBlobMeta get_settable() {
+    return this.settable;
+  }
+
+  public void set_settable(SettableBlobMeta settable) {
+    this.settable = settable;
+  }
+
+  public void unset_settable() {
+    this.settable = null;
+  }
+
+  /** Returns true if field settable is set (has been assigned a value) and false otherwise */
+  public boolean is_set_settable() {
+    return this.settable != null;
+  }
+
+  public void set_settable_isSet(boolean value) {
+    if (!value) {
+      this.settable = null;
+    }
+  }
+
+  public long get_version() {
+    return this.version;
+  }
+
+  public void set_version(long version) {
+    this.version = version;
+    set_version_isSet(true);
+  }
+
+  public void unset_version() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return EncodingUtils.testBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  public void set_version_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VERSION_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SETTABLE:
+      if (value == null) {
+        unset_settable();
+      } else {
+        set_settable((SettableBlobMeta)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SETTABLE:
+      return get_settable();
+
+    case VERSION:
+      return get_version();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SETTABLE:
+      return is_set_settable();
+    case VERSION:
+      return is_set_version();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ReadableBlobMeta)
+      return this.equals((ReadableBlobMeta)that);
+    return false;
+  }
+
+  public boolean equals(ReadableBlobMeta that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_settable = true && this.is_set_settable();
+    boolean that_present_settable = true && that.is_set_settable();
+    if (this_present_settable || that_present_settable) {
+      if (!(this_present_settable && that_present_settable))
+        return false;
+      if (!this.settable.equals(that.settable))
+        return false;
+    }
+
+    boolean this_present_version = true;
+    boolean that_present_version = true;
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (this.version != that.version)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_settable = true && (is_set_settable());
+    list.add(present_settable);
+    if (present_settable)
+      list.add(settable);
+
+    boolean present_version = true;
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ReadableBlobMeta other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_settable()).compareTo(other.is_set_settable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_settable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.settable, other.settable);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ReadableBlobMeta(");
+    boolean first = true;
+
+    sb.append("settable:");
+    if (this.settable == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.settable);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("version:");
+    sb.append(this.version);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_settable()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'settable' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (settable != null) {
+      settable.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ReadableBlobMetaStandardSchemeFactory implements SchemeFactory {
+    public ReadableBlobMetaStandardScheme getScheme() {
+      return new ReadableBlobMetaStandardScheme();
+    }
+  }
+
+  private static class ReadableBlobMetaStandardScheme extends StandardScheme<ReadableBlobMeta> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SETTABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.settable = new SettableBlobMeta();
+              struct.settable.read(iprot);
+              struct.set_settable_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.version = iprot.readI64();
+              struct.set_version_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.settable != null) {
+        oprot.writeFieldBegin(SETTABLE_FIELD_DESC);
+        struct.settable.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(VERSION_FIELD_DESC);
+      oprot.writeI64(struct.version);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ReadableBlobMetaTupleSchemeFactory implements SchemeFactory {
+    public ReadableBlobMetaTupleScheme getScheme() {
+      return new ReadableBlobMetaTupleScheme();
+    }
+  }
+
+  private static class ReadableBlobMetaTupleScheme extends TupleScheme<ReadableBlobMeta> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.settable.write(oprot);
+      oprot.writeI64(struct.version);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.settable = new SettableBlobMeta();
+      struct.settable.read(iprot);
+      struct.set_settable_isSet(true);
+      struct.version = iprot.readI64();
+      struct.set_version_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java b/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java
new file mode 100644
index 0000000..bbb8f91
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java
@@ -0,0 +1,664 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOptions, RebalanceOptions._Fields>, java.io.Serializable, Cloneable, Comparable<RebalanceOptions> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RebalanceOptions");
+
+  private static final org.apache.thrift.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("wait_secs", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.MAP, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new RebalanceOptionsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new RebalanceOptionsTupleSchemeFactory());
+  }
+
+  private int wait_secs; // optional
+  private int num_workers; // optional
+  private Map<String,Integer> num_executors; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    WAIT_SECS((short)1, "wait_secs"),
+    NUM_WORKERS((short)2, "num_workers"),
+    NUM_EXECUTORS((short)3, "num_executors");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // WAIT_SECS
+          return WAIT_SECS;
+        case 2: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 3: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __WAIT_SECS_ISSET_ID = 0;
+  private static final int __NUM_WORKERS_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.WAIT_SECS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift.meta_data.FieldMetaData("wait_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RebalanceOptions.class, metaDataMap);
+  }
+
+  public RebalanceOptions() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public RebalanceOptions(RebalanceOptions other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.wait_secs = other.wait_secs;
+    this.num_workers = other.num_workers;
+    if (other.is_set_num_executors()) {
+      Map<String,Integer> __this__num_executors = new HashMap<String,Integer>(other.num_executors);
+      this.num_executors = __this__num_executors;
+    }
+  }
+
+  public RebalanceOptions deepCopy() {
+    return new RebalanceOptions(this);
+  }
+
+  @Override
+  public void clear() {
+    set_wait_secs_isSet(false);
+    this.wait_secs = 0;
+    set_num_workers_isSet(false);
+    this.num_workers = 0;
+    this.num_executors = null;
+  }
+
+  public int get_wait_secs() {
+    return this.wait_secs;
+  }
+
+  public void set_wait_secs(int wait_secs) {
+    this.wait_secs = wait_secs;
+    set_wait_secs_isSet(true);
+  }
+
+  public void unset_wait_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WAIT_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_wait_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __WAIT_SECS_ISSET_ID);
+  }
+
+  public void set_wait_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WAIT_SECS_ISSET_ID, value);
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_num_executors_size() {
+    return (this.num_executors == null) ? 0 : this.num_executors.size();
+  }
+
+  public void put_to_num_executors(String key, int val) {
+    if (this.num_executors == null) {
+      this.num_executors = new HashMap<String,Integer>();
+    }
+    this.num_executors.put(key, val);
+  }
+
+  public Map<String,Integer> get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(Map<String,Integer> num_executors) {
+    this.num_executors = num_executors;
+  }
+
+  public void unset_num_executors() {
+    this.num_executors = null;
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return this.num_executors != null;
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    if (!value) {
+      this.num_executors = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case WAIT_SECS:
+      if (value == null) {
+        unset_wait_secs();
+      } else {
+        set_wait_secs((Integer)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Map<String,Integer>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case WAIT_SECS:
+      return get_wait_secs();
+
+    case NUM_WORKERS:
+      return get_num_workers();
+
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case WAIT_SECS:
+      return is_set_wait_secs();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof RebalanceOptions)
+      return this.equals((RebalanceOptions)that);
+    return false;
+  }
+
+  public boolean equals(RebalanceOptions that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_wait_secs = true && this.is_set_wait_secs();
+    boolean that_present_wait_secs = true && that.is_set_wait_secs();
+    if (this_present_wait_secs || that_present_wait_secs) {
+      if (!(this_present_wait_secs && that_present_wait_secs))
+        return false;
+      if (this.wait_secs != that.wait_secs)
+        return false;
+    }
+
+    boolean this_present_num_workers = true && this.is_set_num_workers();
+    boolean that_present_num_workers = true && that.is_set_num_workers();
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (!this.num_executors.equals(that.num_executors))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_wait_secs = true && (is_set_wait_secs());
+    list.add(present_wait_secs);
+    if (present_wait_secs)
+      list.add(wait_secs);
+
+    boolean present_num_workers = true && (is_set_num_workers());
+    list.add(present_num_workers);
+    if (present_num_workers)
+      list.add(num_workers);
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(RebalanceOptions other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(other.is_set_wait_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_wait_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.wait_secs, other.wait_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("RebalanceOptions(");
+    boolean first = true;
+
+    if (is_set_wait_secs()) {
+      sb.append("wait_secs:");
+      sb.append(this.wait_secs);
+      first = false;
+    }
+    if (is_set_num_workers()) {
+      if (!first) sb.append(", ");
+      sb.append("num_workers:");
+      sb.append(this.num_workers);
+      first = false;
+    }
+    if (is_set_num_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("num_executors:");
+      if (this.num_executors == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.num_executors);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class RebalanceOptionsStandardSchemeFactory implements SchemeFactory {
+    public RebalanceOptionsStandardScheme getScheme() {
+      return new RebalanceOptionsStandardScheme();
+    }
+  }
+
+  private static class RebalanceOptionsStandardScheme extends StandardScheme<RebalanceOptions> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, RebalanceOptions struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // WAIT_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.wait_secs = iprot.readI32();
+              struct.set_wait_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NUM_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_workers = iprot.readI32();
+              struct.set_num_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map538 = iprot.readMapBegin();
+                struct.num_executors = new HashMap<String,Integer>(2*_map538.size);
+                String _key539;
+                int _val540;
+                for (int _i541 = 0; _i541 < _map538.size; ++_i541)
+                {
+                  _key539 = iprot.readString();
+                  _val540 = iprot.readI32();
+                  struct.num_executors.put(_key539, _val540);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, RebalanceOptions struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_wait_secs()) {
+        oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC);
+        oprot.writeI32(struct.wait_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_workers()) {
+        oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+        oprot.writeI32(struct.num_workers);
+        oprot.writeFieldEnd();
+      }
+      if (struct.num_executors != null) {
+        if (struct.is_set_num_executors()) {
+          oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.num_executors.size()));
+            for (Map.Entry<String, Integer> _iter542 : struct.num_executors.entrySet())
+            {
+              oprot.writeString(_iter542.getKey());
+              oprot.writeI32(_iter542.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class RebalanceOptionsTupleSchemeFactory implements SchemeFactory {
+    public RebalanceOptionsTupleScheme getScheme() {
+      return new RebalanceOptionsTupleScheme();
+    }
+  }
+
+  private static class RebalanceOptionsTupleScheme extends TupleScheme<RebalanceOptions> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_wait_secs()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_num_workers()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_num_executors()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_wait_secs()) {
+        oprot.writeI32(struct.wait_secs);
+      }
+      if (struct.is_set_num_workers()) {
+        oprot.writeI32(struct.num_workers);
+      }
+      if (struct.is_set_num_executors()) {
+        {
+          oprot.writeI32(struct.num_executors.size());
+          for (Map.Entry<String, Integer> _iter543 : struct.num_executors.entrySet())
+          {
+            oprot.writeString(_iter543.getKey());
+            oprot.writeI32(_iter543.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.wait_secs = iprot.readI32();
+        struct.set_wait_secs_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.num_workers = iprot.readI32();
+        struct.set_num_workers_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TMap _map544 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+          struct.num_executors = new HashMap<String,Integer>(2*_map544.size);
+          String _key545;
+          int _val546;
+          for (int _i547 = 0; _i547 < _map544.size; ++_i547)
+          {
+            _key545 = iprot.readString();
+            _val546 = iprot.readI32();
+            struct.num_executors.put(_key545, _val546);
+          }
+        }
+        struct.set_num_executors_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java b/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java
new file mode 100644
index 0000000..349e045
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java
@@ -0,0 +1,567 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SettableBlobMeta implements org.apache.thrift.TBase<SettableBlobMeta, SettableBlobMeta._Fields>, java.io.Serializable, Cloneable, Comparable<SettableBlobMeta> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SettableBlobMeta");
+
+  private static final org.apache.thrift.protocol.TField ACL_FIELD_DESC = new org.apache.thrift.protocol.TField("acl", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField REPLICATION_FACTOR_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_factor", org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SettableBlobMetaStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SettableBlobMetaTupleSchemeFactory());
+  }
+
+  private List<AccessControl> acl; // required
+  private int replication_factor; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ACL((short)1, "acl"),
+    REPLICATION_FACTOR((short)2, "replication_factor");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACL
+          return ACL;
+        case 2: // REPLICATION_FACTOR
+          return REPLICATION_FACTOR;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __REPLICATION_FACTOR_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.REPLICATION_FACTOR};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACL, new org.apache.thrift.meta_data.FieldMetaData("acl", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AccessControl.class))));
+    tmpMap.put(_Fields.REPLICATION_FACTOR, new org.apache.thrift.meta_data.FieldMetaData("replication_factor", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SettableBlobMeta.class, metaDataMap);
+  }
+
+  public SettableBlobMeta() {
+  }
+
+  public SettableBlobMeta(
+    List<AccessControl> acl)
+  {
+    this();
+    this.acl = acl;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SettableBlobMeta(SettableBlobMeta other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_acl()) {
+      List<AccessControl> __this__acl = new ArrayList<AccessControl>(other.acl.size());
+      for (AccessControl other_element : other.acl) {
+        __this__acl.add(new AccessControl(other_element));
+      }
+      this.acl = __this__acl;
+    }
+    this.replication_factor = other.replication_factor;
+  }
+
+  public SettableBlobMeta deepCopy() {
+    return new SettableBlobMeta(this);
+  }
+
+  @Override
+  public void clear() {
+    this.acl = null;
+    set_replication_factor_isSet(false);
+    this.replication_factor = 0;
+  }
+
+  public int get_acl_size() {
+    return (this.acl == null) ? 0 : this.acl.size();
+  }
+
+  public java.util.Iterator<AccessControl> get_acl_iterator() {
+    return (this.acl == null) ? null : this.acl.iterator();
+  }
+
+  public void add_to_acl(AccessControl elem) {
+    if (this.acl == null) {
+      this.acl = new ArrayList<AccessControl>();
+    }
+    this.acl.add(elem);
+  }
+
+  public List<AccessControl> get_acl() {
+    return this.acl;
+  }
+
+  public void set_acl(List<AccessControl> acl) {
+    this.acl = acl;
+  }
+
+  public void unset_acl() {
+    this.acl = null;
+  }
+
+  /** Returns true if field acl is set (has been assigned a value) and false otherwise */
+  public boolean is_set_acl() {
+    return this.acl != null;
+  }
+
+  public void set_acl_isSet(boolean value) {
+    if (!value) {
+      this.acl = null;
+    }
+  }
+
+  public int get_replication_factor() {
+    return this.replication_factor;
+  }
+
+  public void set_replication_factor(int replication_factor) {
+    this.replication_factor = replication_factor;
+    set_replication_factor_isSet(true);
+  }
+
+  public void unset_replication_factor() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_FACTOR_ISSET_ID);
+  }
+
+  /** Returns true if field replication_factor is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_factor() {
+    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_FACTOR_ISSET_ID);
+  }
+
+  public void set_replication_factor_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_FACTOR_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACL:
+      if (value == null) {
+        unset_acl();
+      } else {
+        set_acl((List<AccessControl>)value);
+      }
+      break;
+
+    case REPLICATION_FACTOR:
+      if (value == null) {
+        unset_replication_factor();
+      } else {
+        set_replication_factor((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACL:
+      return get_acl();
+
+    case REPLICATION_FACTOR:
+      return get_replication_factor();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ACL:
+      return is_set_acl();
+    case REPLICATION_FACTOR:
+      return is_set_replication_factor();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SettableBlobMeta)
+      return this.equals((SettableBlobMeta)that);
+    return false;
+  }
+
+  public boolean equals(SettableBlobMeta that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_acl = true && this.is_set_acl();
+    boolean that_present_acl = true && that.is_set_acl();
+    if (this_present_acl || that_present_acl) {
+      if (!(this_present_acl && that_present_acl))
+        return false;
+      if (!this.acl.equals(that.acl))
+        return false;
+    }
+
+    boolean this_present_replication_factor = true && this.is_set_replication_factor();
+    boolean that_present_replication_factor = true && that.is_set_replication_factor();
+    if (this_present_replication_factor || that_present_replication_factor) {
+      if (!(this_present_replication_factor && that_present_replication_factor))
+        return false;
+      if (this.replication_factor != that.replication_factor)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_acl = true && (is_set_acl());
+    list.add(present_acl);
+    if (present_acl)
+      list.add(acl);
+
+    boolean present_replication_factor = true && (is_set_replication_factor());
+    list.add(present_replication_factor);
+    if (present_replication_factor)
+      list.add(replication_factor);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SettableBlobMeta other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_acl()).compareTo(other.is_set_acl());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acl()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acl, other.acl);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_replication_factor()).compareTo(other.is_set_replication_factor());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_factor()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_factor, other.replication_factor);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SettableBlobMeta(");
+    boolean first = true;
+
+    sb.append("acl:");
+    if (this.acl == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.acl);
+    }
+    first = false;
+    if (is_set_replication_factor()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_factor:");
+      sb.append(this.replication_factor);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_acl()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'acl' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SettableBlobMetaStandardSchemeFactory implements SchemeFactory {
+    public SettableBlobMetaStandardScheme getScheme() {
+      return new SettableBlobMetaStandardScheme();
+    }
+  }
+
+  private static class SettableBlobMetaStandardScheme extends StandardScheme<SettableBlobMeta> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ACL
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list558 = iprot.readListBegin();
+                struct.acl = new ArrayList<AccessControl>(_list558.size);
+                AccessControl _elem559;
+                for (int _i560 = 0; _i560 < _list558.size; ++_i560)
+                {
+                  _elem559 = new AccessControl();
+                  _elem559.read(iprot);
+                  struct.acl.add(_elem559);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_acl_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // REPLICATION_FACTOR
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.replication_factor = iprot.readI32();
+              struct.set_replication_factor_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.acl != null) {
+        oprot.writeFieldBegin(ACL_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.acl.size()));
+          for (AccessControl _iter561 : struct.acl)
+          {
+            _iter561.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_replication_factor()) {
+        oprot.writeFieldBegin(REPLICATION_FACTOR_FIELD_DESC);
+        oprot.writeI32(struct.replication_factor);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SettableBlobMetaTupleSchemeFactory implements SchemeFactory {
+    public SettableBlobMetaTupleScheme getScheme() {
+      return new SettableBlobMetaTupleScheme();
+    }
+  }
+
+  private static class SettableBlobMetaTupleScheme extends TupleScheme<SettableBlobMeta> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.acl.size());
+        for (AccessControl _iter562 : struct.acl)
+        {
+          _iter562.write(oprot);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_replication_factor()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_replication_factor()) {
+        oprot.writeI32(struct.replication_factor);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.acl = new ArrayList<AccessControl>(_list563.size);
+        AccessControl _elem564;
+        for (int _i565 = 0; _i565 < _list563.size; ++_i565)
+        {
+          _elem564 = new AccessControl();
+          _elem564.read(iprot);
+          struct.acl.add(_elem564);
+        }
+      }
+      struct.set_acl_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.replication_factor = iprot.readI32();
+        struct.set_replication_factor_isSet(true);
+      }
+    }
+  }
+
+}
+


[27/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ExecutorStats.java b/storm-client/src/jvm/org/apache/storm/generated/ExecutorStats.java
new file mode 100644
index 0000000..bbb8e7a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ExecutorStats.java
@@ -0,0 +1,915 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, ExecutorStats._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorStats");
+
+  private static final org.apache.thrift.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("emitted", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("transferred", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField SPECIFIC_FIELD_DESC = new org.apache.thrift.protocol.TField("specific", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField RATE_FIELD_DESC = new org.apache.thrift.protocol.TField("rate", org.apache.thrift.protocol.TType.DOUBLE, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ExecutorStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ExecutorStatsTupleSchemeFactory());
+  }
+
+  private Map<String,Map<String,Long>> emitted; // required
+  private Map<String,Map<String,Long>> transferred; // required
+  private ExecutorSpecificStats specific; // required
+  private double rate; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EMITTED((short)1, "emitted"),
+    TRANSFERRED((short)2, "transferred"),
+    SPECIFIC((short)3, "specific"),
+    RATE((short)4, "rate");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EMITTED
+          return EMITTED;
+        case 2: // TRANSFERRED
+          return TRANSFERRED;
+        case 3: // SPECIFIC
+          return SPECIFIC;
+        case 4: // RATE
+          return RATE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __RATE_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EMITTED, new org.apache.thrift.meta_data.FieldMetaData("emitted", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
+    tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("transferred", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
+    tmpMap.put(_Fields.SPECIFIC, new org.apache.thrift.meta_data.FieldMetaData("specific", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSpecificStats.class)));
+    tmpMap.put(_Fields.RATE, new org.apache.thrift.meta_data.FieldMetaData("rate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorStats.class, metaDataMap);
+  }
+
+  public ExecutorStats() {
+  }
+
+  public ExecutorStats(
+    Map<String,Map<String,Long>> emitted,
+    Map<String,Map<String,Long>> transferred,
+    ExecutorSpecificStats specific,
+    double rate)
+  {
+    this();
+    this.emitted = emitted;
+    this.transferred = transferred;
+    this.specific = specific;
+    this.rate = rate;
+    set_rate_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ExecutorStats(ExecutorStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_emitted()) {
+      Map<String,Map<String,Long>> __this__emitted = new HashMap<String,Map<String,Long>>(other.emitted.size());
+      for (Map.Entry<String, Map<String,Long>> other_element : other.emitted.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<String,Long> other_element_value = other_element.getValue();
+
+        String __this__emitted_copy_key = other_element_key;
+
+        Map<String,Long> __this__emitted_copy_value = new HashMap<String,Long>(other_element_value);
+
+        __this__emitted.put(__this__emitted_copy_key, __this__emitted_copy_value);
+      }
+      this.emitted = __this__emitted;
+    }
+    if (other.is_set_transferred()) {
+      Map<String,Map<String,Long>> __this__transferred = new HashMap<String,Map<String,Long>>(other.transferred.size());
+      for (Map.Entry<String, Map<String,Long>> other_element : other.transferred.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<String,Long> other_element_value = other_element.getValue();
+
+        String __this__transferred_copy_key = other_element_key;
+
+        Map<String,Long> __this__transferred_copy_value = new HashMap<String,Long>(other_element_value);
+
+        __this__transferred.put(__this__transferred_copy_key, __this__transferred_copy_value);
+      }
+      this.transferred = __this__transferred;
+    }
+    if (other.is_set_specific()) {
+      this.specific = new ExecutorSpecificStats(other.specific);
+    }
+    this.rate = other.rate;
+  }
+
+  public ExecutorStats deepCopy() {
+    return new ExecutorStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.emitted = null;
+    this.transferred = null;
+    this.specific = null;
+    set_rate_isSet(false);
+    this.rate = 0.0;
+  }
+
+  public int get_emitted_size() {
+    return (this.emitted == null) ? 0 : this.emitted.size();
+  }
+
+  public void put_to_emitted(String key, Map<String,Long> val) {
+    if (this.emitted == null) {
+      this.emitted = new HashMap<String,Map<String,Long>>();
+    }
+    this.emitted.put(key, val);
+  }
+
+  public Map<String,Map<String,Long>> get_emitted() {
+    return this.emitted;
+  }
+
+  public void set_emitted(Map<String,Map<String,Long>> emitted) {
+    this.emitted = emitted;
+  }
+
+  public void unset_emitted() {
+    this.emitted = null;
+  }
+
+  /** Returns true if field emitted is set (has been assigned a value) and false otherwise */
+  public boolean is_set_emitted() {
+    return this.emitted != null;
+  }
+
+  public void set_emitted_isSet(boolean value) {
+    if (!value) {
+      this.emitted = null;
+    }
+  }
+
+  public int get_transferred_size() {
+    return (this.transferred == null) ? 0 : this.transferred.size();
+  }
+
+  public void put_to_transferred(String key, Map<String,Long> val) {
+    if (this.transferred == null) {
+      this.transferred = new HashMap<String,Map<String,Long>>();
+    }
+    this.transferred.put(key, val);
+  }
+
+  public Map<String,Map<String,Long>> get_transferred() {
+    return this.transferred;
+  }
+
+  public void set_transferred(Map<String,Map<String,Long>> transferred) {
+    this.transferred = transferred;
+  }
+
+  public void unset_transferred() {
+    this.transferred = null;
+  }
+
+  /** Returns true if field transferred is set (has been assigned a value) and false otherwise */
+  public boolean is_set_transferred() {
+    return this.transferred != null;
+  }
+
+  public void set_transferred_isSet(boolean value) {
+    if (!value) {
+      this.transferred = null;
+    }
+  }
+
+  public ExecutorSpecificStats get_specific() {
+    return this.specific;
+  }
+
+  public void set_specific(ExecutorSpecificStats specific) {
+    this.specific = specific;
+  }
+
+  public void unset_specific() {
+    this.specific = null;
+  }
+
+  /** Returns true if field specific is set (has been assigned a value) and false otherwise */
+  public boolean is_set_specific() {
+    return this.specific != null;
+  }
+
+  public void set_specific_isSet(boolean value) {
+    if (!value) {
+      this.specific = null;
+    }
+  }
+
+  public double get_rate() {
+    return this.rate;
+  }
+
+  public void set_rate(double rate) {
+    this.rate = rate;
+    set_rate_isSet(true);
+  }
+
+  public void unset_rate() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RATE_ISSET_ID);
+  }
+
+  /** Returns true if field rate is set (has been assigned a value) and false otherwise */
+  public boolean is_set_rate() {
+    return EncodingUtils.testBit(__isset_bitfield, __RATE_ISSET_ID);
+  }
+
+  public void set_rate_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RATE_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EMITTED:
+      if (value == null) {
+        unset_emitted();
+      } else {
+        set_emitted((Map<String,Map<String,Long>>)value);
+      }
+      break;
+
+    case TRANSFERRED:
+      if (value == null) {
+        unset_transferred();
+      } else {
+        set_transferred((Map<String,Map<String,Long>>)value);
+      }
+      break;
+
+    case SPECIFIC:
+      if (value == null) {
+        unset_specific();
+      } else {
+        set_specific((ExecutorSpecificStats)value);
+      }
+      break;
+
+    case RATE:
+      if (value == null) {
+        unset_rate();
+      } else {
+        set_rate((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EMITTED:
+      return get_emitted();
+
+    case TRANSFERRED:
+      return get_transferred();
+
+    case SPECIFIC:
+      return get_specific();
+
+    case RATE:
+      return get_rate();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EMITTED:
+      return is_set_emitted();
+    case TRANSFERRED:
+      return is_set_transferred();
+    case SPECIFIC:
+      return is_set_specific();
+    case RATE:
+      return is_set_rate();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ExecutorStats)
+      return this.equals((ExecutorStats)that);
+    return false;
+  }
+
+  public boolean equals(ExecutorStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_emitted = true && this.is_set_emitted();
+    boolean that_present_emitted = true && that.is_set_emitted();
+    if (this_present_emitted || that_present_emitted) {
+      if (!(this_present_emitted && that_present_emitted))
+        return false;
+      if (!this.emitted.equals(that.emitted))
+        return false;
+    }
+
+    boolean this_present_transferred = true && this.is_set_transferred();
+    boolean that_present_transferred = true && that.is_set_transferred();
+    if (this_present_transferred || that_present_transferred) {
+      if (!(this_present_transferred && that_present_transferred))
+        return false;
+      if (!this.transferred.equals(that.transferred))
+        return false;
+    }
+
+    boolean this_present_specific = true && this.is_set_specific();
+    boolean that_present_specific = true && that.is_set_specific();
+    if (this_present_specific || that_present_specific) {
+      if (!(this_present_specific && that_present_specific))
+        return false;
+      if (!this.specific.equals(that.specific))
+        return false;
+    }
+
+    boolean this_present_rate = true;
+    boolean that_present_rate = true;
+    if (this_present_rate || that_present_rate) {
+      if (!(this_present_rate && that_present_rate))
+        return false;
+      if (this.rate != that.rate)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_emitted = true && (is_set_emitted());
+    list.add(present_emitted);
+    if (present_emitted)
+      list.add(emitted);
+
+    boolean present_transferred = true && (is_set_transferred());
+    list.add(present_transferred);
+    if (present_transferred)
+      list.add(transferred);
+
+    boolean present_specific = true && (is_set_specific());
+    list.add(present_specific);
+    if (present_specific)
+      list.add(specific);
+
+    boolean present_rate = true;
+    list.add(present_rate);
+    if (present_rate)
+      list.add(rate);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ExecutorStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_emitted()).compareTo(other.is_set_emitted());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_emitted()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.emitted, other.emitted);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_transferred()).compareTo(other.is_set_transferred());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_transferred()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.transferred, other.transferred);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_specific()).compareTo(other.is_set_specific());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_specific()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.specific, other.specific);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_rate()).compareTo(other.is_set_rate());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_rate()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rate, other.rate);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ExecutorStats(");
+    boolean first = true;
+
+    sb.append("emitted:");
+    if (this.emitted == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.emitted);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("transferred:");
+    if (this.transferred == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.transferred);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("specific:");
+    if (this.specific == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.specific);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("rate:");
+    sb.append(this.rate);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_emitted()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'emitted' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_transferred()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'transferred' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_specific()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'specific' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_rate()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'rate' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ExecutorStatsStandardSchemeFactory implements SchemeFactory {
+    public ExecutorStatsStandardScheme getScheme() {
+      return new ExecutorStatsStandardScheme();
+    }
+  }
+
+  private static class ExecutorStatsStandardScheme extends StandardScheme<ExecutorStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EMITTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map292 = iprot.readMapBegin();
+                struct.emitted = new HashMap<String,Map<String,Long>>(2*_map292.size);
+                String _key293;
+                Map<String,Long> _val294;
+                for (int _i295 = 0; _i295 < _map292.size; ++_i295)
+                {
+                  _key293 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map296 = iprot.readMapBegin();
+                    _val294 = new HashMap<String,Long>(2*_map296.size);
+                    String _key297;
+                    long _val298;
+                    for (int _i299 = 0; _i299 < _map296.size; ++_i299)
+                    {
+                      _key297 = iprot.readString();
+                      _val298 = iprot.readI64();
+                      _val294.put(_key297, _val298);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.emitted.put(_key293, _val294);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_emitted_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TRANSFERRED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map300 = iprot.readMapBegin();
+                struct.transferred = new HashMap<String,Map<String,Long>>(2*_map300.size);
+                String _key301;
+                Map<String,Long> _val302;
+                for (int _i303 = 0; _i303 < _map300.size; ++_i303)
+                {
+                  _key301 = iprot.readString();
+                  {
+                    org.apache.thrift.protocol.TMap _map304 = iprot.readMapBegin();
+                    _val302 = new HashMap<String,Long>(2*_map304.size);
+                    String _key305;
+                    long _val306;
+                    for (int _i307 = 0; _i307 < _map304.size; ++_i307)
+                    {
+                      _key305 = iprot.readString();
+                      _val306 = iprot.readI64();
+                      _val302.put(_key305, _val306);
+                    }
+                    iprot.readMapEnd();
+                  }
+                  struct.transferred.put(_key301, _val302);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_transferred_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // SPECIFIC
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.specific = new ExecutorSpecificStats();
+              struct.specific.read(iprot);
+              struct.set_specific_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // RATE
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.rate = iprot.readDouble();
+              struct.set_rate_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.emitted != null) {
+        oprot.writeFieldBegin(EMITTED_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.emitted.size()));
+          for (Map.Entry<String, Map<String,Long>> _iter308 : struct.emitted.entrySet())
+          {
+            oprot.writeString(_iter308.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter308.getValue().size()));
+              for (Map.Entry<String, Long> _iter309 : _iter308.getValue().entrySet())
+              {
+                oprot.writeString(_iter309.getKey());
+                oprot.writeI64(_iter309.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.transferred != null) {
+        oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.transferred.size()));
+          for (Map.Entry<String, Map<String,Long>> _iter310 : struct.transferred.entrySet())
+          {
+            oprot.writeString(_iter310.getKey());
+            {
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter310.getValue().size()));
+              for (Map.Entry<String, Long> _iter311 : _iter310.getValue().entrySet())
+              {
+                oprot.writeString(_iter311.getKey());
+                oprot.writeI64(_iter311.getValue());
+              }
+              oprot.writeMapEnd();
+            }
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.specific != null) {
+        oprot.writeFieldBegin(SPECIFIC_FIELD_DESC);
+        struct.specific.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(RATE_FIELD_DESC);
+      oprot.writeDouble(struct.rate);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ExecutorStatsTupleSchemeFactory implements SchemeFactory {
+    public ExecutorStatsTupleScheme getScheme() {
+      return new ExecutorStatsTupleScheme();
+    }
+  }
+
+  private static class ExecutorStatsTupleScheme extends TupleScheme<ExecutorStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.emitted.size());
+        for (Map.Entry<String, Map<String,Long>> _iter312 : struct.emitted.entrySet())
+        {
+          oprot.writeString(_iter312.getKey());
+          {
+            oprot.writeI32(_iter312.getValue().size());
+            for (Map.Entry<String, Long> _iter313 : _iter312.getValue().entrySet())
+            {
+              oprot.writeString(_iter313.getKey());
+              oprot.writeI64(_iter313.getValue());
+            }
+          }
+        }
+      }
+      {
+        oprot.writeI32(struct.transferred.size());
+        for (Map.Entry<String, Map<String,Long>> _iter314 : struct.transferred.entrySet())
+        {
+          oprot.writeString(_iter314.getKey());
+          {
+            oprot.writeI32(_iter314.getValue().size());
+            for (Map.Entry<String, Long> _iter315 : _iter314.getValue().entrySet())
+            {
+              oprot.writeString(_iter315.getKey());
+              oprot.writeI64(_iter315.getValue());
+            }
+          }
+        }
+      }
+      struct.specific.write(oprot);
+      oprot.writeDouble(struct.rate);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map316 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.emitted = new HashMap<String,Map<String,Long>>(2*_map316.size);
+        String _key317;
+        Map<String,Long> _val318;
+        for (int _i319 = 0; _i319 < _map316.size; ++_i319)
+        {
+          _key317 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map320 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val318 = new HashMap<String,Long>(2*_map320.size);
+            String _key321;
+            long _val322;
+            for (int _i323 = 0; _i323 < _map320.size; ++_i323)
+            {
+              _key321 = iprot.readString();
+              _val322 = iprot.readI64();
+              _val318.put(_key321, _val322);
+            }
+          }
+          struct.emitted.put(_key317, _val318);
+        }
+      }
+      struct.set_emitted_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map324 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.transferred = new HashMap<String,Map<String,Long>>(2*_map324.size);
+        String _key325;
+        Map<String,Long> _val326;
+        for (int _i327 = 0; _i327 < _map324.size; ++_i327)
+        {
+          _key325 = iprot.readString();
+          {
+            org.apache.thrift.protocol.TMap _map328 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val326 = new HashMap<String,Long>(2*_map328.size);
+            String _key329;
+            long _val330;
+            for (int _i331 = 0; _i331 < _map328.size; ++_i331)
+            {
+              _key329 = iprot.readString();
+              _val330 = iprot.readI64();
+              _val326.put(_key329, _val330);
+            }
+          }
+          struct.transferred.put(_key325, _val326);
+        }
+      }
+      struct.set_transferred_isSet(true);
+      struct.specific = new ExecutorSpecificStats();
+      struct.specific.read(iprot);
+      struct.set_specific_isSet(true);
+      struct.rate = iprot.readDouble();
+      struct.set_rate_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ExecutorSummary.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ExecutorSummary.java b/storm-client/src/jvm/org/apache/storm/generated/ExecutorSummary.java
new file mode 100644
index 0000000..ef8182e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ExecutorSummary.java
@@ -0,0 +1,922 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ExecutorSummary implements org.apache.thrift.TBase<ExecutorSummary, ExecutorSummary._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorSummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorSummary");
+
+  private static final org.apache.thrift.protocol.TField EXECUTOR_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_info", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("stats", org.apache.thrift.protocol.TType.STRUCT, (short)7);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ExecutorSummaryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ExecutorSummaryTupleSchemeFactory());
+  }
+
+  private ExecutorInfo executor_info; // required
+  private String component_id; // required
+  private String host; // required
+  private int port; // required
+  private int uptime_secs; // required
+  private ExecutorStats stats; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXECUTOR_INFO((short)1, "executor_info"),
+    COMPONENT_ID((short)2, "component_id"),
+    HOST((short)3, "host"),
+    PORT((short)4, "port"),
+    UPTIME_SECS((short)5, "uptime_secs"),
+    STATS((short)7, "stats");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXECUTOR_INFO
+          return EXECUTOR_INFO;
+        case 2: // COMPONENT_ID
+          return COMPONENT_ID;
+        case 3: // HOST
+          return HOST;
+        case 4: // PORT
+          return PORT;
+        case 5: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 7: // STATS
+          return STATS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PORT_ISSET_ID = 0;
+  private static final int __UPTIME_SECS_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.STATS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXECUTOR_INFO, new org.apache.thrift.meta_data.FieldMetaData("executor_info", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class)));
+    tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.STATS, new org.apache.thrift.meta_data.FieldMetaData("stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorSummary.class, metaDataMap);
+  }
+
+  public ExecutorSummary() {
+  }
+
+  public ExecutorSummary(
+    ExecutorInfo executor_info,
+    String component_id,
+    String host,
+    int port,
+    int uptime_secs)
+  {
+    this();
+    this.executor_info = executor_info;
+    this.component_id = component_id;
+    this.host = host;
+    this.port = port;
+    set_port_isSet(true);
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ExecutorSummary(ExecutorSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_executor_info()) {
+      this.executor_info = new ExecutorInfo(other.executor_info);
+    }
+    if (other.is_set_component_id()) {
+      this.component_id = other.component_id;
+    }
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.port = other.port;
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_stats()) {
+      this.stats = new ExecutorStats(other.stats);
+    }
+  }
+
+  public ExecutorSummary deepCopy() {
+    return new ExecutorSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.executor_info = null;
+    this.component_id = null;
+    this.host = null;
+    set_port_isSet(false);
+    this.port = 0;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    this.stats = null;
+  }
+
+  public ExecutorInfo get_executor_info() {
+    return this.executor_info;
+  }
+
+  public void set_executor_info(ExecutorInfo executor_info) {
+    this.executor_info = executor_info;
+  }
+
+  public void unset_executor_info() {
+    this.executor_info = null;
+  }
+
+  /** Returns true if field executor_info is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_info() {
+    return this.executor_info != null;
+  }
+
+  public void set_executor_info_isSet(boolean value) {
+    if (!value) {
+      this.executor_info = null;
+    }
+  }
+
+  public String get_component_id() {
+    return this.component_id;
+  }
+
+  public void set_component_id(String component_id) {
+    this.component_id = component_id;
+  }
+
+  public void unset_component_id() {
+    this.component_id = null;
+  }
+
+  /** Returns true if field component_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_id() {
+    return this.component_id != null;
+  }
+
+  public void set_component_id_isSet(boolean value) {
+    if (!value) {
+      this.component_id = null;
+    }
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public ExecutorStats get_stats() {
+    return this.stats;
+  }
+
+  public void set_stats(ExecutorStats stats) {
+    this.stats = stats;
+  }
+
+  public void unset_stats() {
+    this.stats = null;
+  }
+
+  /** Returns true if field stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_stats() {
+    return this.stats != null;
+  }
+
+  public void set_stats_isSet(boolean value) {
+    if (!value) {
+      this.stats = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXECUTOR_INFO:
+      if (value == null) {
+        unset_executor_info();
+      } else {
+        set_executor_info((ExecutorInfo)value);
+      }
+      break;
+
+    case COMPONENT_ID:
+      if (value == null) {
+        unset_component_id();
+      } else {
+        set_component_id((String)value);
+      }
+      break;
+
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case STATS:
+      if (value == null) {
+        unset_stats();
+      } else {
+        set_stats((ExecutorStats)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXECUTOR_INFO:
+      return get_executor_info();
+
+    case COMPONENT_ID:
+      return get_component_id();
+
+    case HOST:
+      return get_host();
+
+    case PORT:
+      return get_port();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case STATS:
+      return get_stats();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXECUTOR_INFO:
+      return is_set_executor_info();
+    case COMPONENT_ID:
+      return is_set_component_id();
+    case HOST:
+      return is_set_host();
+    case PORT:
+      return is_set_port();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case STATS:
+      return is_set_stats();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ExecutorSummary)
+      return this.equals((ExecutorSummary)that);
+    return false;
+  }
+
+  public boolean equals(ExecutorSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_executor_info = true && this.is_set_executor_info();
+    boolean that_present_executor_info = true && that.is_set_executor_info();
+    if (this_present_executor_info || that_present_executor_info) {
+      if (!(this_present_executor_info && that_present_executor_info))
+        return false;
+      if (!this.executor_info.equals(that.executor_info))
+        return false;
+    }
+
+    boolean this_present_component_id = true && this.is_set_component_id();
+    boolean that_present_component_id = true && that.is_set_component_id();
+    if (this_present_component_id || that_present_component_id) {
+      if (!(this_present_component_id && that_present_component_id))
+        return false;
+      if (!this.component_id.equals(that.component_id))
+        return false;
+    }
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_port = true;
+    boolean that_present_port = true;
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_stats = true && this.is_set_stats();
+    boolean that_present_stats = true && that.is_set_stats();
+    if (this_present_stats || that_present_stats) {
+      if (!(this_present_stats && that_present_stats))
+        return false;
+      if (!this.stats.equals(that.stats))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_executor_info = true && (is_set_executor_info());
+    list.add(present_executor_info);
+    if (present_executor_info)
+      list.add(executor_info);
+
+    boolean present_component_id = true && (is_set_component_id());
+    list.add(present_component_id);
+    if (present_component_id)
+      list.add(component_id);
+
+    boolean present_host = true && (is_set_host());
+    list.add(present_host);
+    if (present_host)
+      list.add(host);
+
+    boolean present_port = true;
+    list.add(present_port);
+    if (present_port)
+      list.add(port);
+
+    boolean present_uptime_secs = true;
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_stats = true && (is_set_stats());
+    list.add(present_stats);
+    if (present_stats)
+      list.add(stats);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ExecutorSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_executor_info()).compareTo(other.is_set_executor_info());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_info()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_info, other.executor_info);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(other.is_set_component_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, other.component_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_stats()).compareTo(other.is_set_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, other.stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ExecutorSummary(");
+    boolean first = true;
+
+    sb.append("executor_info:");
+    if (this.executor_info == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executor_info);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("component_id:");
+    if (this.component_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.component_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("host:");
+    if (this.host == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.host);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    sb.append(this.port);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (is_set_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("stats:");
+      if (this.stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.stats);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_executor_info()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executor_info' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_component_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_host()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (executor_info != null) {
+      executor_info.validate();
+    }
+    if (stats != null) {
+      stats.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ExecutorSummaryStandardSchemeFactory implements SchemeFactory {
+    public ExecutorSummaryStandardScheme getScheme() {
+      return new ExecutorSummaryStandardScheme();
+    }
+  }
+
+  private static class ExecutorSummaryStandardScheme extends StandardScheme<ExecutorSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXECUTOR_INFO
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.executor_info = new ExecutorInfo();
+              struct.executor_info.read(iprot);
+              struct.set_executor_info_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMPONENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.component_id = iprot.readString();
+              struct.set_component_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.host = iprot.readString();
+              struct.set_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.port = iprot.readI32();
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.stats = new ExecutorStats();
+              struct.stats.read(iprot);
+              struct.set_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.executor_info != null) {
+        oprot.writeFieldBegin(EXECUTOR_INFO_FIELD_DESC);
+        struct.executor_info.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.component_id != null) {
+        oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+        oprot.writeString(struct.component_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.host != null) {
+        oprot.writeFieldBegin(HOST_FIELD_DESC);
+        oprot.writeString(struct.host);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(PORT_FIELD_DESC);
+      oprot.writeI32(struct.port);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeFieldEnd();
+      if (struct.stats != null) {
+        if (struct.is_set_stats()) {
+          oprot.writeFieldBegin(STATS_FIELD_DESC);
+          struct.stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ExecutorSummaryTupleSchemeFactory implements SchemeFactory {
+    public ExecutorSummaryTupleScheme getScheme() {
+      return new ExecutorSummaryTupleScheme();
+    }
+  }
+
+  private static class ExecutorSummaryTupleScheme extends TupleScheme<ExecutorSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.executor_info.write(oprot);
+      oprot.writeString(struct.component_id);
+      oprot.writeString(struct.host);
+      oprot.writeI32(struct.port);
+      oprot.writeI32(struct.uptime_secs);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_stats()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_stats()) {
+        struct.stats.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.executor_info = new ExecutorInfo();
+      struct.executor_info.read(iprot);
+      struct.set_executor_info_isSet(true);
+      struct.component_id = iprot.readString();
+      struct.set_component_id_isSet(true);
+      struct.host = iprot.readString();
+      struct.set_host_isSet(true);
+      struct.port = iprot.readI32();
+      struct.set_port_isSet(true);
+      struct.uptime_secs = iprot.readI32();
+      struct.set_uptime_secs_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.stats = new ExecutorStats();
+        struct.stats.read(iprot);
+        struct.set_stats_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/GetInfoOptions.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/GetInfoOptions.java b/storm-client/src/jvm/org/apache/storm/generated/GetInfoOptions.java
new file mode 100644
index 0000000..de91563
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/GetInfoOptions.java
@@ -0,0 +1,422 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class GetInfoOptions implements org.apache.thrift.TBase<GetInfoOptions, GetInfoOptions._Fields>, java.io.Serializable, Cloneable, Comparable<GetInfoOptions> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfoOptions");
+
+  private static final org.apache.thrift.protocol.TField NUM_ERR_CHOICE_FIELD_DESC = new org.apache.thrift.protocol.TField("num_err_choice", org.apache.thrift.protocol.TType.I32, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetInfoOptionsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetInfoOptionsTupleSchemeFactory());
+  }
+
+  private NumErrorsChoice num_err_choice; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see NumErrorsChoice
+     */
+    NUM_ERR_CHOICE((short)1, "num_err_choice");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NUM_ERR_CHOICE
+          return NUM_ERR_CHOICE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.NUM_ERR_CHOICE};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NUM_ERR_CHOICE, new org.apache.thrift.meta_data.FieldMetaData("num_err_choice", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, NumErrorsChoice.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfoOptions.class, metaDataMap);
+  }
+
+  public GetInfoOptions() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetInfoOptions(GetInfoOptions other) {
+    if (other.is_set_num_err_choice()) {
+      this.num_err_choice = other.num_err_choice;
+    }
+  }
+
+  public GetInfoOptions deepCopy() {
+    return new GetInfoOptions(this);
+  }
+
+  @Override
+  public void clear() {
+    this.num_err_choice = null;
+  }
+
+  /**
+   * 
+   * @see NumErrorsChoice
+   */
+  public NumErrorsChoice get_num_err_choice() {
+    return this.num_err_choice;
+  }
+
+  /**
+   * 
+   * @see NumErrorsChoice
+   */
+  public void set_num_err_choice(NumErrorsChoice num_err_choice) {
+    this.num_err_choice = num_err_choice;
+  }
+
+  public void unset_num_err_choice() {
+    this.num_err_choice = null;
+  }
+
+  /** Returns true if field num_err_choice is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_err_choice() {
+    return this.num_err_choice != null;
+  }
+
+  public void set_num_err_choice_isSet(boolean value) {
+    if (!value) {
+      this.num_err_choice = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NUM_ERR_CHOICE:
+      if (value == null) {
+        unset_num_err_choice();
+      } else {
+        set_num_err_choice((NumErrorsChoice)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NUM_ERR_CHOICE:
+      return get_num_err_choice();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NUM_ERR_CHOICE:
+      return is_set_num_err_choice();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetInfoOptions)
+      return this.equals((GetInfoOptions)that);
+    return false;
+  }
+
+  public boolean equals(GetInfoOptions that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_num_err_choice = true && this.is_set_num_err_choice();
+    boolean that_present_num_err_choice = true && that.is_set_num_err_choice();
+    if (this_present_num_err_choice || that_present_num_err_choice) {
+      if (!(this_present_num_err_choice && that_present_num_err_choice))
+        return false;
+      if (!this.num_err_choice.equals(that.num_err_choice))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_num_err_choice = true && (is_set_num_err_choice());
+    list.add(present_num_err_choice);
+    if (present_num_err_choice)
+      list.add(num_err_choice.getValue());
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetInfoOptions other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_num_err_choice()).compareTo(other.is_set_num_err_choice());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_err_choice()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_err_choice, other.num_err_choice);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetInfoOptions(");
+    boolean first = true;
+
+    if (is_set_num_err_choice()) {
+      sb.append("num_err_choice:");
+      if (this.num_err_choice == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.num_err_choice);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetInfoOptionsStandardSchemeFactory implements SchemeFactory {
+    public GetInfoOptionsStandardScheme getScheme() {
+      return new GetInfoOptionsStandardScheme();
+    }
+  }
+
+  private static class GetInfoOptionsStandardScheme extends StandardScheme<GetInfoOptions> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfoOptions struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NUM_ERR_CHOICE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_err_choice = org.apache.storm.generated.NumErrorsChoice.findByValue(iprot.readI32());
+              struct.set_num_err_choice_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfoOptions struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.num_err_choice != null) {
+        if (struct.is_set_num_err_choice()) {
+          oprot.writeFieldBegin(NUM_ERR_CHOICE_FIELD_DESC);
+          oprot.writeI32(struct.num_err_choice.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetInfoOptionsTupleSchemeFactory implements SchemeFactory {
+    public GetInfoOptionsTupleScheme getScheme() {
+      return new GetInfoOptionsTupleScheme();
+    }
+  }
+
+  private static class GetInfoOptionsTupleScheme extends TupleScheme<GetInfoOptions> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetInfoOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_num_err_choice()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_num_err_choice()) {
+        oprot.writeI32(struct.num_err_choice.getValue());
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetInfoOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.num_err_choice = org.apache.storm.generated.NumErrorsChoice.findByValue(iprot.readI32());
+        struct.set_num_err_choice_isSet(true);
+      }
+    }
+  }
+
+}
+


[16/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ShellComponent.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ShellComponent.java b/storm-client/src/jvm/org/apache/storm/generated/ShellComponent.java
new file mode 100644
index 0000000..6941cb2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ShellComponent.java
@@ -0,0 +1,516 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ShellComponent implements org.apache.thrift.TBase<ShellComponent, ShellComponent._Fields>, java.io.Serializable, Cloneable, Comparable<ShellComponent> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShellComponent");
+
+  private static final org.apache.thrift.protocol.TField EXECUTION_COMMAND_FIELD_DESC = new org.apache.thrift.protocol.TField("execution_command", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField SCRIPT_FIELD_DESC = new org.apache.thrift.protocol.TField("script", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ShellComponentStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ShellComponentTupleSchemeFactory());
+  }
+
+  private String execution_command; // required
+  private String script; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXECUTION_COMMAND((short)1, "execution_command"),
+    SCRIPT((short)2, "script");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXECUTION_COMMAND
+          return EXECUTION_COMMAND;
+        case 2: // SCRIPT
+          return SCRIPT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXECUTION_COMMAND, new org.apache.thrift.meta_data.FieldMetaData("execution_command", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SCRIPT, new org.apache.thrift.meta_data.FieldMetaData("script", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShellComponent.class, metaDataMap);
+  }
+
+  public ShellComponent() {
+  }
+
+  public ShellComponent(
+    String execution_command,
+    String script)
+  {
+    this();
+    this.execution_command = execution_command;
+    this.script = script;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ShellComponent(ShellComponent other) {
+    if (other.is_set_execution_command()) {
+      this.execution_command = other.execution_command;
+    }
+    if (other.is_set_script()) {
+      this.script = other.script;
+    }
+  }
+
+  public ShellComponent deepCopy() {
+    return new ShellComponent(this);
+  }
+
+  @Override
+  public void clear() {
+    this.execution_command = null;
+    this.script = null;
+  }
+
+  public String get_execution_command() {
+    return this.execution_command;
+  }
+
+  public void set_execution_command(String execution_command) {
+    this.execution_command = execution_command;
+  }
+
+  public void unset_execution_command() {
+    this.execution_command = null;
+  }
+
+  /** Returns true if field execution_command is set (has been assigned a value) and false otherwise */
+  public boolean is_set_execution_command() {
+    return this.execution_command != null;
+  }
+
+  public void set_execution_command_isSet(boolean value) {
+    if (!value) {
+      this.execution_command = null;
+    }
+  }
+
+  public String get_script() {
+    return this.script;
+  }
+
+  public void set_script(String script) {
+    this.script = script;
+  }
+
+  public void unset_script() {
+    this.script = null;
+  }
+
+  /** Returns true if field script is set (has been assigned a value) and false otherwise */
+  public boolean is_set_script() {
+    return this.script != null;
+  }
+
+  public void set_script_isSet(boolean value) {
+    if (!value) {
+      this.script = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXECUTION_COMMAND:
+      if (value == null) {
+        unset_execution_command();
+      } else {
+        set_execution_command((String)value);
+      }
+      break;
+
+    case SCRIPT:
+      if (value == null) {
+        unset_script();
+      } else {
+        set_script((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXECUTION_COMMAND:
+      return get_execution_command();
+
+    case SCRIPT:
+      return get_script();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXECUTION_COMMAND:
+      return is_set_execution_command();
+    case SCRIPT:
+      return is_set_script();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ShellComponent)
+      return this.equals((ShellComponent)that);
+    return false;
+  }
+
+  public boolean equals(ShellComponent that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_execution_command = true && this.is_set_execution_command();
+    boolean that_present_execution_command = true && that.is_set_execution_command();
+    if (this_present_execution_command || that_present_execution_command) {
+      if (!(this_present_execution_command && that_present_execution_command))
+        return false;
+      if (!this.execution_command.equals(that.execution_command))
+        return false;
+    }
+
+    boolean this_present_script = true && this.is_set_script();
+    boolean that_present_script = true && that.is_set_script();
+    if (this_present_script || that_present_script) {
+      if (!(this_present_script && that_present_script))
+        return false;
+      if (!this.script.equals(that.script))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_execution_command = true && (is_set_execution_command());
+    list.add(present_execution_command);
+    if (present_execution_command)
+      list.add(execution_command);
+
+    boolean present_script = true && (is_set_script());
+    list.add(present_script);
+    if (present_script)
+      list.add(script);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ShellComponent other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_execution_command()).compareTo(other.is_set_execution_command());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_execution_command()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.execution_command, other.execution_command);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_script()).compareTo(other.is_set_script());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_script()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.script, other.script);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ShellComponent(");
+    boolean first = true;
+
+    sb.append("execution_command:");
+    if (this.execution_command == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.execution_command);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("script:");
+    if (this.script == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.script);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ShellComponentStandardSchemeFactory implements SchemeFactory {
+    public ShellComponentStandardScheme getScheme() {
+      return new ShellComponentStandardScheme();
+    }
+  }
+
+  private static class ShellComponentStandardScheme extends StandardScheme<ShellComponent> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ShellComponent struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXECUTION_COMMAND
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.execution_command = iprot.readString();
+              struct.set_execution_command_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SCRIPT
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.script = iprot.readString();
+              struct.set_script_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ShellComponent struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.execution_command != null) {
+        oprot.writeFieldBegin(EXECUTION_COMMAND_FIELD_DESC);
+        oprot.writeString(struct.execution_command);
+        oprot.writeFieldEnd();
+      }
+      if (struct.script != null) {
+        oprot.writeFieldBegin(SCRIPT_FIELD_DESC);
+        oprot.writeString(struct.script);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ShellComponentTupleSchemeFactory implements SchemeFactory {
+    public ShellComponentTupleScheme getScheme() {
+      return new ShellComponentTupleScheme();
+    }
+  }
+
+  private static class ShellComponentTupleScheme extends TupleScheme<ShellComponent> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ShellComponent struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_execution_command()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_script()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_execution_command()) {
+        oprot.writeString(struct.execution_command);
+      }
+      if (struct.is_set_script()) {
+        oprot.writeString(struct.script);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ShellComponent struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.execution_command = iprot.readString();
+        struct.set_execution_command_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.script = iprot.readString();
+        struct.set_script_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SpecificAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SpecificAggregateStats.java b/storm-client/src/jvm/org/apache/storm/generated/SpecificAggregateStats.java
new file mode 100644
index 0000000..22bff71
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SpecificAggregateStats.java
@@ -0,0 +1,387 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class SpecificAggregateStats extends org.apache.thrift.TUnion<SpecificAggregateStats, SpecificAggregateStats._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpecificAggregateStats");
+  private static final org.apache.thrift.protocol.TField BOLT_FIELD_DESC = new org.apache.thrift.protocol.TField("bolt", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField SPOUT_FIELD_DESC = new org.apache.thrift.protocol.TField("spout", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    BOLT((short)1, "bolt"),
+    SPOUT((short)2, "spout");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // BOLT
+          return BOLT;
+        case 2: // SPOUT
+          return SPOUT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.BOLT, new org.apache.thrift.meta_data.FieldMetaData("bolt", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, BoltAggregateStats.class)));
+    tmpMap.put(_Fields.SPOUT, new org.apache.thrift.meta_data.FieldMetaData("spout", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpoutAggregateStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpecificAggregateStats.class, metaDataMap);
+  }
+
+  public SpecificAggregateStats() {
+    super();
+  }
+
+  public SpecificAggregateStats(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public SpecificAggregateStats(SpecificAggregateStats other) {
+    super(other);
+  }
+  public SpecificAggregateStats deepCopy() {
+    return new SpecificAggregateStats(this);
+  }
+
+  public static SpecificAggregateStats bolt(BoltAggregateStats value) {
+    SpecificAggregateStats x = new SpecificAggregateStats();
+    x.set_bolt(value);
+    return x;
+  }
+
+  public static SpecificAggregateStats spout(SpoutAggregateStats value) {
+    SpecificAggregateStats x = new SpecificAggregateStats();
+    x.set_spout(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case BOLT:
+        if (value instanceof BoltAggregateStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type BoltAggregateStats for field 'bolt', but got " + value.getClass().getSimpleName());
+      case SPOUT:
+        if (value instanceof SpoutAggregateStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type SpoutAggregateStats for field 'spout', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case BOLT:
+          if (field.type == BOLT_FIELD_DESC.type) {
+            BoltAggregateStats bolt;
+            bolt = new BoltAggregateStats();
+            bolt.read(iprot);
+            return bolt;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SPOUT:
+          if (field.type == SPOUT_FIELD_DESC.type) {
+            SpoutAggregateStats spout;
+            spout = new SpoutAggregateStats();
+            spout.read(iprot);
+            return spout;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case BOLT:
+        BoltAggregateStats bolt = (BoltAggregateStats)value_;
+        bolt.write(oprot);
+        return;
+      case SPOUT:
+        SpoutAggregateStats spout = (SpoutAggregateStats)value_;
+        spout.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case BOLT:
+          BoltAggregateStats bolt;
+          bolt = new BoltAggregateStats();
+          bolt.read(iprot);
+          return bolt;
+        case SPOUT:
+          SpoutAggregateStats spout;
+          spout = new SpoutAggregateStats();
+          spout.read(iprot);
+          return spout;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case BOLT:
+        BoltAggregateStats bolt = (BoltAggregateStats)value_;
+        bolt.write(oprot);
+        return;
+      case SPOUT:
+        SpoutAggregateStats spout = (SpoutAggregateStats)value_;
+        spout.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case BOLT:
+        return BOLT_FIELD_DESC;
+      case SPOUT:
+        return SPOUT_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public BoltAggregateStats get_bolt() {
+    if (getSetField() == _Fields.BOLT) {
+      return (BoltAggregateStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'bolt' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_bolt(BoltAggregateStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.BOLT;
+    value_ = value;
+  }
+
+  public SpoutAggregateStats get_spout() {
+    if (getSetField() == _Fields.SPOUT) {
+      return (SpoutAggregateStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'spout' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_spout(SpoutAggregateStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SPOUT;
+    value_ = value;
+  }
+
+  public boolean is_set_bolt() {
+    return setField_ == _Fields.BOLT;
+  }
+
+
+  public boolean is_set_spout() {
+    return setField_ == _Fields.SPOUT;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof SpecificAggregateStats) {
+      return equals((SpecificAggregateStats)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(SpecificAggregateStats other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(SpecificAggregateStats other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SpoutAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SpoutAggregateStats.java b/storm-client/src/jvm/org/apache/storm/generated/SpoutAggregateStats.java
new file mode 100644
index 0000000..712bc35
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SpoutAggregateStats.java
@@ -0,0 +1,407 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SpoutAggregateStats implements org.apache.thrift.TBase<SpoutAggregateStats, SpoutAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutAggregateStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutAggregateStats");
+
+  private static final org.apache.thrift.protocol.TField COMPLETE_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("complete_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SpoutAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SpoutAggregateStatsTupleSchemeFactory());
+  }
+
+  private double complete_latency_ms; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    COMPLETE_LATENCY_MS((short)1, "complete_latency_ms");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // COMPLETE_LATENCY_MS
+          return COMPLETE_LATENCY_MS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __COMPLETE_LATENCY_MS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.COMPLETE_LATENCY_MS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.COMPLETE_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("complete_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpoutAggregateStats.class, metaDataMap);
+  }
+
+  public SpoutAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SpoutAggregateStats(SpoutAggregateStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.complete_latency_ms = other.complete_latency_ms;
+  }
+
+  public SpoutAggregateStats deepCopy() {
+    return new SpoutAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    set_complete_latency_ms_isSet(false);
+    this.complete_latency_ms = 0.0;
+  }
+
+  public double get_complete_latency_ms() {
+    return this.complete_latency_ms;
+  }
+
+  public void set_complete_latency_ms(double complete_latency_ms) {
+    this.complete_latency_ms = complete_latency_ms;
+    set_complete_latency_ms_isSet(true);
+  }
+
+  public void unset_complete_latency_ms() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID);
+  }
+
+  /** Returns true if field complete_latency_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_complete_latency_ms() {
+    return EncodingUtils.testBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID);
+  }
+
+  public void set_complete_latency_ms_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case COMPLETE_LATENCY_MS:
+      if (value == null) {
+        unset_complete_latency_ms();
+      } else {
+        set_complete_latency_ms((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case COMPLETE_LATENCY_MS:
+      return get_complete_latency_ms();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case COMPLETE_LATENCY_MS:
+      return is_set_complete_latency_ms();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SpoutAggregateStats)
+      return this.equals((SpoutAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(SpoutAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_complete_latency_ms = true && this.is_set_complete_latency_ms();
+    boolean that_present_complete_latency_ms = true && that.is_set_complete_latency_ms();
+    if (this_present_complete_latency_ms || that_present_complete_latency_ms) {
+      if (!(this_present_complete_latency_ms && that_present_complete_latency_ms))
+        return false;
+      if (this.complete_latency_ms != that.complete_latency_ms)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_complete_latency_ms = true && (is_set_complete_latency_ms());
+    list.add(present_complete_latency_ms);
+    if (present_complete_latency_ms)
+      list.add(complete_latency_ms);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SpoutAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_complete_latency_ms()).compareTo(other.is_set_complete_latency_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_complete_latency_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.complete_latency_ms, other.complete_latency_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SpoutAggregateStats(");
+    boolean first = true;
+
+    if (is_set_complete_latency_ms()) {
+      sb.append("complete_latency_ms:");
+      sb.append(this.complete_latency_ms);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SpoutAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public SpoutAggregateStatsStandardScheme getScheme() {
+      return new SpoutAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class SpoutAggregateStatsStandardScheme extends StandardScheme<SpoutAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // COMPLETE_LATENCY_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.complete_latency_ms = iprot.readDouble();
+              struct.set_complete_latency_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_complete_latency_ms()) {
+        oprot.writeFieldBegin(COMPLETE_LATENCY_MS_FIELD_DESC);
+        oprot.writeDouble(struct.complete_latency_ms);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SpoutAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public SpoutAggregateStatsTupleScheme getScheme() {
+      return new SpoutAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class SpoutAggregateStatsTupleScheme extends TupleScheme<SpoutAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_complete_latency_ms()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_complete_latency_ms()) {
+        oprot.writeDouble(struct.complete_latency_ms);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.complete_latency_ms = iprot.readDouble();
+        struct.set_complete_latency_ms_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SpoutSpec.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SpoutSpec.java b/storm-client/src/jvm/org/apache/storm/generated/SpoutSpec.java
new file mode 100644
index 0000000..bb7919e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SpoutSpec.java
@@ -0,0 +1,514 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SpoutSpec implements org.apache.thrift.TBase<SpoutSpec, SpoutSpec._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutSpec> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutSpec");
+
+  private static final org.apache.thrift.protocol.TField SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("spout_object", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift.protocol.TField("common", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SpoutSpecStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SpoutSpecTupleSchemeFactory());
+  }
+
+  private ComponentObject spout_object; // required
+  private ComponentCommon common; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SPOUT_OBJECT((short)1, "spout_object"),
+    COMMON((short)2, "common");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SPOUT_OBJECT
+          return SPOUT_OBJECT;
+        case 2: // COMMON
+          return COMMON;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SPOUT_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("spout_object", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentObject.class)));
+    tmpMap.put(_Fields.COMMON, new org.apache.thrift.meta_data.FieldMetaData("common", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentCommon.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpoutSpec.class, metaDataMap);
+  }
+
+  public SpoutSpec() {
+  }
+
+  public SpoutSpec(
+    ComponentObject spout_object,
+    ComponentCommon common)
+  {
+    this();
+    this.spout_object = spout_object;
+    this.common = common;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SpoutSpec(SpoutSpec other) {
+    if (other.is_set_spout_object()) {
+      this.spout_object = new ComponentObject(other.spout_object);
+    }
+    if (other.is_set_common()) {
+      this.common = new ComponentCommon(other.common);
+    }
+  }
+
+  public SpoutSpec deepCopy() {
+    return new SpoutSpec(this);
+  }
+
+  @Override
+  public void clear() {
+    this.spout_object = null;
+    this.common = null;
+  }
+
+  public ComponentObject get_spout_object() {
+    return this.spout_object;
+  }
+
+  public void set_spout_object(ComponentObject spout_object) {
+    this.spout_object = spout_object;
+  }
+
+  public void unset_spout_object() {
+    this.spout_object = null;
+  }
+
+  /** Returns true if field spout_object is set (has been assigned a value) and false otherwise */
+  public boolean is_set_spout_object() {
+    return this.spout_object != null;
+  }
+
+  public void set_spout_object_isSet(boolean value) {
+    if (!value) {
+      this.spout_object = null;
+    }
+  }
+
+  public ComponentCommon get_common() {
+    return this.common;
+  }
+
+  public void set_common(ComponentCommon common) {
+    this.common = common;
+  }
+
+  public void unset_common() {
+    this.common = null;
+  }
+
+  /** Returns true if field common is set (has been assigned a value) and false otherwise */
+  public boolean is_set_common() {
+    return this.common != null;
+  }
+
+  public void set_common_isSet(boolean value) {
+    if (!value) {
+      this.common = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SPOUT_OBJECT:
+      if (value == null) {
+        unset_spout_object();
+      } else {
+        set_spout_object((ComponentObject)value);
+      }
+      break;
+
+    case COMMON:
+      if (value == null) {
+        unset_common();
+      } else {
+        set_common((ComponentCommon)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SPOUT_OBJECT:
+      return get_spout_object();
+
+    case COMMON:
+      return get_common();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SPOUT_OBJECT:
+      return is_set_spout_object();
+    case COMMON:
+      return is_set_common();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SpoutSpec)
+      return this.equals((SpoutSpec)that);
+    return false;
+  }
+
+  public boolean equals(SpoutSpec that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_spout_object = true && this.is_set_spout_object();
+    boolean that_present_spout_object = true && that.is_set_spout_object();
+    if (this_present_spout_object || that_present_spout_object) {
+      if (!(this_present_spout_object && that_present_spout_object))
+        return false;
+      if (!this.spout_object.equals(that.spout_object))
+        return false;
+    }
+
+    boolean this_present_common = true && this.is_set_common();
+    boolean that_present_common = true && that.is_set_common();
+    if (this_present_common || that_present_common) {
+      if (!(this_present_common && that_present_common))
+        return false;
+      if (!this.common.equals(that.common))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_spout_object = true && (is_set_spout_object());
+    list.add(present_spout_object);
+    if (present_spout_object)
+      list.add(spout_object);
+
+    boolean present_common = true && (is_set_common());
+    list.add(present_common);
+    if (present_common)
+      list.add(common);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SpoutSpec other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_spout_object()).compareTo(other.is_set_spout_object());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_spout_object()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.spout_object, other.spout_object);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common()).compareTo(other.is_set_common());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common, other.common);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SpoutSpec(");
+    boolean first = true;
+
+    sb.append("spout_object:");
+    if (this.spout_object == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.spout_object);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("common:");
+    if (this.common == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.common);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_spout_object()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'spout_object' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_common()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (common != null) {
+      common.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SpoutSpecStandardSchemeFactory implements SchemeFactory {
+    public SpoutSpecStandardScheme getScheme() {
+      return new SpoutSpecStandardScheme();
+    }
+  }
+
+  private static class SpoutSpecStandardScheme extends StandardScheme<SpoutSpec> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SpoutSpec struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SPOUT_OBJECT
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.spout_object = new ComponentObject();
+              struct.spout_object.read(iprot);
+              struct.set_spout_object_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMMON
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.common = new ComponentCommon();
+              struct.common.read(iprot);
+              struct.set_common_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SpoutSpec struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.spout_object != null) {
+        oprot.writeFieldBegin(SPOUT_OBJECT_FIELD_DESC);
+        struct.spout_object.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.common != null) {
+        oprot.writeFieldBegin(COMMON_FIELD_DESC);
+        struct.common.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SpoutSpecTupleSchemeFactory implements SchemeFactory {
+    public SpoutSpecTupleScheme getScheme() {
+      return new SpoutSpecTupleScheme();
+    }
+  }
+
+  private static class SpoutSpecTupleScheme extends TupleScheme<SpoutSpec> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SpoutSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.spout_object.write(oprot);
+      struct.common.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SpoutSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.spout_object = new ComponentObject();
+      struct.spout_object.read(iprot);
+      struct.set_spout_object_isSet(true);
+      struct.common = new ComponentCommon();
+      struct.common.read(iprot);
+      struct.set_common_isSet(true);
+    }
+  }
+
+}
+


[12/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java b/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java
new file mode 100644
index 0000000..44003c2
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java
@@ -0,0 +1,1446 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo");
+
+  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ASSIGNMENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("assignment_id", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField USED_PORTS_FIELD_DESC = new org.apache.thrift.protocol.TField("used_ports", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField META_FIELD_DESC = new org.apache.thrift.protocol.TField("meta", org.apache.thrift.protocol.TType.LIST, (short)5);
+  private static final org.apache.thrift.protocol.TField SCHEDULER_META_FIELD_DESC = new org.apache.thrift.protocol.TField("scheduler_meta", org.apache.thrift.protocol.TType.MAP, (short)6);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I64, (short)7);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField RESOURCES_MAP_FIELD_DESC = new org.apache.thrift.protocol.TField("resources_map", org.apache.thrift.protocol.TType.MAP, (short)9);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SupervisorInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SupervisorInfoTupleSchemeFactory());
+  }
+
+  private long time_secs; // required
+  private String hostname; // required
+  private String assignment_id; // optional
+  private List<Long> used_ports; // optional
+  private List<Long> meta; // optional
+  private Map<String,String> scheduler_meta; // optional
+  private long uptime_secs; // optional
+  private String version; // optional
+  private Map<String,Double> resources_map; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TIME_SECS((short)1, "time_secs"),
+    HOSTNAME((short)2, "hostname"),
+    ASSIGNMENT_ID((short)3, "assignment_id"),
+    USED_PORTS((short)4, "used_ports"),
+    META((short)5, "meta"),
+    SCHEDULER_META((short)6, "scheduler_meta"),
+    UPTIME_SECS((short)7, "uptime_secs"),
+    VERSION((short)8, "version"),
+    RESOURCES_MAP((short)9, "resources_map");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TIME_SECS
+          return TIME_SECS;
+        case 2: // HOSTNAME
+          return HOSTNAME;
+        case 3: // ASSIGNMENT_ID
+          return ASSIGNMENT_ID;
+        case 4: // USED_PORTS
+          return USED_PORTS;
+        case 5: // META
+          return META;
+        case 6: // SCHEDULER_META
+          return SCHEDULER_META;
+        case 7: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 8: // VERSION
+          return VERSION;
+        case 9: // RESOURCES_MAP
+          return RESOURCES_MAP;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_SECS_ISSET_ID = 0;
+  private static final int __UPTIME_SECS_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ASSIGNMENT_ID,_Fields.USED_PORTS,_Fields.META,_Fields.SCHEDULER_META,_Fields.UPTIME_SECS,_Fields.VERSION,_Fields.RESOURCES_MAP};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ASSIGNMENT_ID, new org.apache.thrift.meta_data.FieldMetaData("assignment_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.USED_PORTS, new org.apache.thrift.meta_data.FieldMetaData("used_ports", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.META, new org.apache.thrift.meta_data.FieldMetaData("meta", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.SCHEDULER_META, new org.apache.thrift.meta_data.FieldMetaData("scheduler_meta", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.RESOURCES_MAP, new org.apache.thrift.meta_data.FieldMetaData("resources_map", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorInfo.class, metaDataMap);
+  }
+
+  public SupervisorInfo() {
+  }
+
+  public SupervisorInfo(
+    long time_secs,
+    String hostname)
+  {
+    this();
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+    this.hostname = hostname;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SupervisorInfo(SupervisorInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.time_secs = other.time_secs;
+    if (other.is_set_hostname()) {
+      this.hostname = other.hostname;
+    }
+    if (other.is_set_assignment_id()) {
+      this.assignment_id = other.assignment_id;
+    }
+    if (other.is_set_used_ports()) {
+      List<Long> __this__used_ports = new ArrayList<Long>(other.used_ports);
+      this.used_ports = __this__used_ports;
+    }
+    if (other.is_set_meta()) {
+      List<Long> __this__meta = new ArrayList<Long>(other.meta);
+      this.meta = __this__meta;
+    }
+    if (other.is_set_scheduler_meta()) {
+      Map<String,String> __this__scheduler_meta = new HashMap<String,String>(other.scheduler_meta);
+      this.scheduler_meta = __this__scheduler_meta;
+    }
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_version()) {
+      this.version = other.version;
+    }
+    if (other.is_set_resources_map()) {
+      Map<String,Double> __this__resources_map = new HashMap<String,Double>(other.resources_map);
+      this.resources_map = __this__resources_map;
+    }
+  }
+
+  public SupervisorInfo deepCopy() {
+    return new SupervisorInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    set_time_secs_isSet(false);
+    this.time_secs = 0;
+    this.hostname = null;
+    this.assignment_id = null;
+    this.used_ports = null;
+    this.meta = null;
+    this.scheduler_meta = null;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    this.version = null;
+    this.resources_map = null;
+  }
+
+  public long get_time_secs() {
+    return this.time_secs;
+  }
+
+  public void set_time_secs(long time_secs) {
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  public void unset_time_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  public void set_time_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_hostname() {
+    return this.hostname;
+  }
+
+  public void set_hostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  public void unset_hostname() {
+    this.hostname = null;
+  }
+
+  /** Returns true if field hostname is set (has been assigned a value) and false otherwise */
+  public boolean is_set_hostname() {
+    return this.hostname != null;
+  }
+
+  public void set_hostname_isSet(boolean value) {
+    if (!value) {
+      this.hostname = null;
+    }
+  }
+
+  public String get_assignment_id() {
+    return this.assignment_id;
+  }
+
+  public void set_assignment_id(String assignment_id) {
+    this.assignment_id = assignment_id;
+  }
+
+  public void unset_assignment_id() {
+    this.assignment_id = null;
+  }
+
+  /** Returns true if field assignment_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assignment_id() {
+    return this.assignment_id != null;
+  }
+
+  public void set_assignment_id_isSet(boolean value) {
+    if (!value) {
+      this.assignment_id = null;
+    }
+  }
+
+  public int get_used_ports_size() {
+    return (this.used_ports == null) ? 0 : this.used_ports.size();
+  }
+
+  public java.util.Iterator<Long> get_used_ports_iterator() {
+    return (this.used_ports == null) ? null : this.used_ports.iterator();
+  }
+
+  public void add_to_used_ports(long elem) {
+    if (this.used_ports == null) {
+      this.used_ports = new ArrayList<Long>();
+    }
+    this.used_ports.add(elem);
+  }
+
+  public List<Long> get_used_ports() {
+    return this.used_ports;
+  }
+
+  public void set_used_ports(List<Long> used_ports) {
+    this.used_ports = used_ports;
+  }
+
+  public void unset_used_ports() {
+    this.used_ports = null;
+  }
+
+  /** Returns true if field used_ports is set (has been assigned a value) and false otherwise */
+  public boolean is_set_used_ports() {
+    return this.used_ports != null;
+  }
+
+  public void set_used_ports_isSet(boolean value) {
+    if (!value) {
+      this.used_ports = null;
+    }
+  }
+
+  public int get_meta_size() {
+    return (this.meta == null) ? 0 : this.meta.size();
+  }
+
+  public java.util.Iterator<Long> get_meta_iterator() {
+    return (this.meta == null) ? null : this.meta.iterator();
+  }
+
+  public void add_to_meta(long elem) {
+    if (this.meta == null) {
+      this.meta = new ArrayList<Long>();
+    }
+    this.meta.add(elem);
+  }
+
+  public List<Long> get_meta() {
+    return this.meta;
+  }
+
+  public void set_meta(List<Long> meta) {
+    this.meta = meta;
+  }
+
+  public void unset_meta() {
+    this.meta = null;
+  }
+
+  /** Returns true if field meta is set (has been assigned a value) and false otherwise */
+  public boolean is_set_meta() {
+    return this.meta != null;
+  }
+
+  public void set_meta_isSet(boolean value) {
+    if (!value) {
+      this.meta = null;
+    }
+  }
+
+  public int get_scheduler_meta_size() {
+    return (this.scheduler_meta == null) ? 0 : this.scheduler_meta.size();
+  }
+
+  public void put_to_scheduler_meta(String key, String val) {
+    if (this.scheduler_meta == null) {
+      this.scheduler_meta = new HashMap<String,String>();
+    }
+    this.scheduler_meta.put(key, val);
+  }
+
+  public Map<String,String> get_scheduler_meta() {
+    return this.scheduler_meta;
+  }
+
+  public void set_scheduler_meta(Map<String,String> scheduler_meta) {
+    this.scheduler_meta = scheduler_meta;
+  }
+
+  public void unset_scheduler_meta() {
+    this.scheduler_meta = null;
+  }
+
+  /** Returns true if field scheduler_meta is set (has been assigned a value) and false otherwise */
+  public boolean is_set_scheduler_meta() {
+    return this.scheduler_meta != null;
+  }
+
+  public void set_scheduler_meta_isSet(boolean value) {
+    if (!value) {
+      this.scheduler_meta = null;
+    }
+  }
+
+  public long get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(long uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_version() {
+    return this.version;
+  }
+
+  public void set_version(String version) {
+    this.version = version;
+  }
+
+  public void unset_version() {
+    this.version = null;
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return this.version != null;
+  }
+
+  public void set_version_isSet(boolean value) {
+    if (!value) {
+      this.version = null;
+    }
+  }
+
+  public int get_resources_map_size() {
+    return (this.resources_map == null) ? 0 : this.resources_map.size();
+  }
+
+  public void put_to_resources_map(String key, double val) {
+    if (this.resources_map == null) {
+      this.resources_map = new HashMap<String,Double>();
+    }
+    this.resources_map.put(key, val);
+  }
+
+  public Map<String,Double> get_resources_map() {
+    return this.resources_map;
+  }
+
+  public void set_resources_map(Map<String,Double> resources_map) {
+    this.resources_map = resources_map;
+  }
+
+  public void unset_resources_map() {
+    this.resources_map = null;
+  }
+
+  /** Returns true if field resources_map is set (has been assigned a value) and false otherwise */
+  public boolean is_set_resources_map() {
+    return this.resources_map != null;
+  }
+
+  public void set_resources_map_isSet(boolean value) {
+    if (!value) {
+      this.resources_map = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TIME_SECS:
+      if (value == null) {
+        unset_time_secs();
+      } else {
+        set_time_secs((Long)value);
+      }
+      break;
+
+    case HOSTNAME:
+      if (value == null) {
+        unset_hostname();
+      } else {
+        set_hostname((String)value);
+      }
+      break;
+
+    case ASSIGNMENT_ID:
+      if (value == null) {
+        unset_assignment_id();
+      } else {
+        set_assignment_id((String)value);
+      }
+      break;
+
+    case USED_PORTS:
+      if (value == null) {
+        unset_used_ports();
+      } else {
+        set_used_ports((List<Long>)value);
+      }
+      break;
+
+    case META:
+      if (value == null) {
+        unset_meta();
+      } else {
+        set_meta((List<Long>)value);
+      }
+      break;
+
+    case SCHEDULER_META:
+      if (value == null) {
+        unset_scheduler_meta();
+      } else {
+        set_scheduler_meta((Map<String,String>)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Long)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((String)value);
+      }
+      break;
+
+    case RESOURCES_MAP:
+      if (value == null) {
+        unset_resources_map();
+      } else {
+        set_resources_map((Map<String,Double>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TIME_SECS:
+      return get_time_secs();
+
+    case HOSTNAME:
+      return get_hostname();
+
+    case ASSIGNMENT_ID:
+      return get_assignment_id();
+
+    case USED_PORTS:
+      return get_used_ports();
+
+    case META:
+      return get_meta();
+
+    case SCHEDULER_META:
+      return get_scheduler_meta();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case VERSION:
+      return get_version();
+
+    case RESOURCES_MAP:
+      return get_resources_map();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TIME_SECS:
+      return is_set_time_secs();
+    case HOSTNAME:
+      return is_set_hostname();
+    case ASSIGNMENT_ID:
+      return is_set_assignment_id();
+    case USED_PORTS:
+      return is_set_used_ports();
+    case META:
+      return is_set_meta();
+    case SCHEDULER_META:
+      return is_set_scheduler_meta();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case VERSION:
+      return is_set_version();
+    case RESOURCES_MAP:
+      return is_set_resources_map();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SupervisorInfo)
+      return this.equals((SupervisorInfo)that);
+    return false;
+  }
+
+  public boolean equals(SupervisorInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_time_secs = true;
+    boolean that_present_time_secs = true;
+    if (this_present_time_secs || that_present_time_secs) {
+      if (!(this_present_time_secs && that_present_time_secs))
+        return false;
+      if (this.time_secs != that.time_secs)
+        return false;
+    }
+
+    boolean this_present_hostname = true && this.is_set_hostname();
+    boolean that_present_hostname = true && that.is_set_hostname();
+    if (this_present_hostname || that_present_hostname) {
+      if (!(this_present_hostname && that_present_hostname))
+        return false;
+      if (!this.hostname.equals(that.hostname))
+        return false;
+    }
+
+    boolean this_present_assignment_id = true && this.is_set_assignment_id();
+    boolean that_present_assignment_id = true && that.is_set_assignment_id();
+    if (this_present_assignment_id || that_present_assignment_id) {
+      if (!(this_present_assignment_id && that_present_assignment_id))
+        return false;
+      if (!this.assignment_id.equals(that.assignment_id))
+        return false;
+    }
+
+    boolean this_present_used_ports = true && this.is_set_used_ports();
+    boolean that_present_used_ports = true && that.is_set_used_ports();
+    if (this_present_used_ports || that_present_used_ports) {
+      if (!(this_present_used_ports && that_present_used_ports))
+        return false;
+      if (!this.used_ports.equals(that.used_ports))
+        return false;
+    }
+
+    boolean this_present_meta = true && this.is_set_meta();
+    boolean that_present_meta = true && that.is_set_meta();
+    if (this_present_meta || that_present_meta) {
+      if (!(this_present_meta && that_present_meta))
+        return false;
+      if (!this.meta.equals(that.meta))
+        return false;
+    }
+
+    boolean this_present_scheduler_meta = true && this.is_set_scheduler_meta();
+    boolean that_present_scheduler_meta = true && that.is_set_scheduler_meta();
+    if (this_present_scheduler_meta || that_present_scheduler_meta) {
+      if (!(this_present_scheduler_meta && that_present_scheduler_meta))
+        return false;
+      if (!this.scheduler_meta.equals(that.scheduler_meta))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true && this.is_set_uptime_secs();
+    boolean that_present_uptime_secs = true && that.is_set_uptime_secs();
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_version = true && this.is_set_version();
+    boolean that_present_version = true && that.is_set_version();
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (!this.version.equals(that.version))
+        return false;
+    }
+
+    boolean this_present_resources_map = true && this.is_set_resources_map();
+    boolean that_present_resources_map = true && that.is_set_resources_map();
+    if (this_present_resources_map || that_present_resources_map) {
+      if (!(this_present_resources_map && that_present_resources_map))
+        return false;
+      if (!this.resources_map.equals(that.resources_map))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_time_secs = true;
+    list.add(present_time_secs);
+    if (present_time_secs)
+      list.add(time_secs);
+
+    boolean present_hostname = true && (is_set_hostname());
+    list.add(present_hostname);
+    if (present_hostname)
+      list.add(hostname);
+
+    boolean present_assignment_id = true && (is_set_assignment_id());
+    list.add(present_assignment_id);
+    if (present_assignment_id)
+      list.add(assignment_id);
+
+    boolean present_used_ports = true && (is_set_used_ports());
+    list.add(present_used_ports);
+    if (present_used_ports)
+      list.add(used_ports);
+
+    boolean present_meta = true && (is_set_meta());
+    list.add(present_meta);
+    if (present_meta)
+      list.add(meta);
+
+    boolean present_scheduler_meta = true && (is_set_scheduler_meta());
+    list.add(present_scheduler_meta);
+    if (present_scheduler_meta)
+      list.add(scheduler_meta);
+
+    boolean present_uptime_secs = true && (is_set_uptime_secs());
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_version = true && (is_set_version());
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    boolean present_resources_map = true && (is_set_resources_map());
+    list.add(present_resources_map);
+    if (present_resources_map)
+      list.add(resources_map);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SupervisorInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_hostname()).compareTo(other.is_set_hostname());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_hostname()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, other.hostname);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assignment_id()).compareTo(other.is_set_assignment_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assignment_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assignment_id, other.assignment_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_used_ports()).compareTo(other.is_set_used_ports());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_used_ports()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_ports, other.used_ports);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_meta()).compareTo(other.is_set_meta());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_meta()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.meta, other.meta);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_scheduler_meta()).compareTo(other.is_set_scheduler_meta());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_scheduler_meta()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scheduler_meta, other.scheduler_meta);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_resources_map()).compareTo(other.is_set_resources_map());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_resources_map()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resources_map, other.resources_map);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SupervisorInfo(");
+    boolean first = true;
+
+    sb.append("time_secs:");
+    sb.append(this.time_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("hostname:");
+    if (this.hostname == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.hostname);
+    }
+    first = false;
+    if (is_set_assignment_id()) {
+      if (!first) sb.append(", ");
+      sb.append("assignment_id:");
+      if (this.assignment_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.assignment_id);
+      }
+      first = false;
+    }
+    if (is_set_used_ports()) {
+      if (!first) sb.append(", ");
+      sb.append("used_ports:");
+      if (this.used_ports == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.used_ports);
+      }
+      first = false;
+    }
+    if (is_set_meta()) {
+      if (!first) sb.append(", ");
+      sb.append("meta:");
+      if (this.meta == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.meta);
+      }
+      first = false;
+    }
+    if (is_set_scheduler_meta()) {
+      if (!first) sb.append(", ");
+      sb.append("scheduler_meta:");
+      if (this.scheduler_meta == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.scheduler_meta);
+      }
+      first = false;
+    }
+    if (is_set_uptime_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("uptime_secs:");
+      sb.append(this.uptime_secs);
+      first = false;
+    }
+    if (is_set_version()) {
+      if (!first) sb.append(", ");
+      sb.append("version:");
+      if (this.version == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.version);
+      }
+      first = false;
+    }
+    if (is_set_resources_map()) {
+      if (!first) sb.append(", ");
+      sb.append("resources_map:");
+      if (this.resources_map == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.resources_map);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_time_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_hostname()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SupervisorInfoStandardSchemeFactory implements SchemeFactory {
+    public SupervisorInfoStandardScheme getScheme() {
+      return new SupervisorInfoStandardScheme();
+    }
+  }
+
+  private static class SupervisorInfoStandardScheme extends StandardScheme<SupervisorInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.time_secs = iprot.readI64();
+              struct.set_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // HOSTNAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.hostname = iprot.readString();
+              struct.set_hostname_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ASSIGNMENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.assignment_id = iprot.readString();
+              struct.set_assignment_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // USED_PORTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
+                struct.used_ports = new ArrayList<Long>(_list574.size);
+                long _elem575;
+                for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+                {
+                  _elem575 = iprot.readI64();
+                  struct.used_ports.add(_elem575);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_used_ports_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // META
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list577 = iprot.readListBegin();
+                struct.meta = new ArrayList<Long>(_list577.size);
+                long _elem578;
+                for (int _i579 = 0; _i579 < _list577.size; ++_i579)
+                {
+                  _elem578 = iprot.readI64();
+                  struct.meta.add(_elem578);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_meta_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // SCHEDULER_META
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map580 = iprot.readMapBegin();
+                struct.scheduler_meta = new HashMap<String,String>(2*_map580.size);
+                String _key581;
+                String _val582;
+                for (int _i583 = 0; _i583 < _map580.size; ++_i583)
+                {
+                  _key581 = iprot.readString();
+                  _val582 = iprot.readString();
+                  struct.scheduler_meta.put(_key581, _val582);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_scheduler_meta_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.uptime_secs = iprot.readI64();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.version = iprot.readString();
+              struct.set_version_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // RESOURCES_MAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map584 = iprot.readMapBegin();
+                struct.resources_map = new HashMap<String,Double>(2*_map584.size);
+                String _key585;
+                double _val586;
+                for (int _i587 = 0; _i587 < _map584.size; ++_i587)
+                {
+                  _key585 = iprot.readString();
+                  _val586 = iprot.readDouble();
+                  struct.resources_map.put(_key585, _val586);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_resources_map_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+      oprot.writeI64(struct.time_secs);
+      oprot.writeFieldEnd();
+      if (struct.hostname != null) {
+        oprot.writeFieldBegin(HOSTNAME_FIELD_DESC);
+        oprot.writeString(struct.hostname);
+        oprot.writeFieldEnd();
+      }
+      if (struct.assignment_id != null) {
+        if (struct.is_set_assignment_id()) {
+          oprot.writeFieldBegin(ASSIGNMENT_ID_FIELD_DESC);
+          oprot.writeString(struct.assignment_id);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.used_ports != null) {
+        if (struct.is_set_used_ports()) {
+          oprot.writeFieldBegin(USED_PORTS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.used_ports.size()));
+            for (long _iter588 : struct.used_ports)
+            {
+              oprot.writeI64(_iter588);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.meta != null) {
+        if (struct.is_set_meta()) {
+          oprot.writeFieldBegin(META_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size()));
+            for (long _iter589 : struct.meta)
+            {
+              oprot.writeI64(_iter589);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.scheduler_meta != null) {
+        if (struct.is_set_scheduler_meta()) {
+          oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.scheduler_meta.size()));
+            for (Map.Entry<String, String> _iter590 : struct.scheduler_meta.entrySet())
+            {
+              oprot.writeString(_iter590.getKey());
+              oprot.writeString(_iter590.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+        oprot.writeI64(struct.uptime_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.version != null) {
+        if (struct.is_set_version()) {
+          oprot.writeFieldBegin(VERSION_FIELD_DESC);
+          oprot.writeString(struct.version);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.resources_map != null) {
+        if (struct.is_set_resources_map()) {
+          oprot.writeFieldBegin(RESOURCES_MAP_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.resources_map.size()));
+            for (Map.Entry<String, Double> _iter591 : struct.resources_map.entrySet())
+            {
+              oprot.writeString(_iter591.getKey());
+              oprot.writeDouble(_iter591.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SupervisorInfoTupleSchemeFactory implements SchemeFactory {
+    public SupervisorInfoTupleScheme getScheme() {
+      return new SupervisorInfoTupleScheme();
+    }
+  }
+
+  private static class SupervisorInfoTupleScheme extends TupleScheme<SupervisorInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.time_secs);
+      oprot.writeString(struct.hostname);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_assignment_id()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_used_ports()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_meta()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_scheduler_meta()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_uptime_secs()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_version()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_resources_map()) {
+        optionals.set(6);
+      }
+      oprot.writeBitSet(optionals, 7);
+      if (struct.is_set_assignment_id()) {
+        oprot.writeString(struct.assignment_id);
+      }
+      if (struct.is_set_used_ports()) {
+        {
+          oprot.writeI32(struct.used_ports.size());
+          for (long _iter592 : struct.used_ports)
+          {
+            oprot.writeI64(_iter592);
+          }
+        }
+      }
+      if (struct.is_set_meta()) {
+        {
+          oprot.writeI32(struct.meta.size());
+          for (long _iter593 : struct.meta)
+          {
+            oprot.writeI64(_iter593);
+          }
+        }
+      }
+      if (struct.is_set_scheduler_meta()) {
+        {
+          oprot.writeI32(struct.scheduler_meta.size());
+          for (Map.Entry<String, String> _iter594 : struct.scheduler_meta.entrySet())
+          {
+            oprot.writeString(_iter594.getKey());
+            oprot.writeString(_iter594.getValue());
+          }
+        }
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeI64(struct.uptime_secs);
+      }
+      if (struct.is_set_version()) {
+        oprot.writeString(struct.version);
+      }
+      if (struct.is_set_resources_map()) {
+        {
+          oprot.writeI32(struct.resources_map.size());
+          for (Map.Entry<String, Double> _iter595 : struct.resources_map.entrySet())
+          {
+            oprot.writeString(_iter595.getKey());
+            oprot.writeDouble(_iter595.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.time_secs = iprot.readI64();
+      struct.set_time_secs_isSet(true);
+      struct.hostname = iprot.readString();
+      struct.set_hostname_isSet(true);
+      BitSet incoming = iprot.readBitSet(7);
+      if (incoming.get(0)) {
+        struct.assignment_id = iprot.readString();
+        struct.set_assignment_id_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list596 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.used_ports = new ArrayList<Long>(_list596.size);
+          long _elem597;
+          for (int _i598 = 0; _i598 < _list596.size; ++_i598)
+          {
+            _elem597 = iprot.readI64();
+            struct.used_ports.add(_elem597);
+          }
+        }
+        struct.set_used_ports_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.meta = new ArrayList<Long>(_list599.size);
+          long _elem600;
+          for (int _i601 = 0; _i601 < _list599.size; ++_i601)
+          {
+            _elem600 = iprot.readI64();
+            struct.meta.add(_elem600);
+          }
+        }
+        struct.set_meta_isSet(true);
+      }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TMap _map602 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.scheduler_meta = new HashMap<String,String>(2*_map602.size);
+          String _key603;
+          String _val604;
+          for (int _i605 = 0; _i605 < _map602.size; ++_i605)
+          {
+            _key603 = iprot.readString();
+            _val604 = iprot.readString();
+            struct.scheduler_meta.put(_key603, _val604);
+          }
+        }
+        struct.set_scheduler_meta_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.uptime_secs = iprot.readI64();
+        struct.set_uptime_secs_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.version = iprot.readString();
+        struct.set_version_isSet(true);
+      }
+      if (incoming.get(6)) {
+        {
+          org.apache.thrift.protocol.TMap _map606 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.resources_map = new HashMap<String,Double>(2*_map606.size);
+          String _key607;
+          double _val608;
+          for (int _i609 = 0; _i609 < _map606.size; ++_i609)
+          {
+            _key607 = iprot.readString();
+            _val608 = iprot.readDouble();
+            struct.resources_map.put(_key607, _val608);
+          }
+        }
+        struct.set_resources_map_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SupervisorPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SupervisorPageInfo.java b/storm-client/src/jvm/org/apache/storm/generated/SupervisorPageInfo.java
new file mode 100644
index 0000000..c6bafd9
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorPageInfo.java
@@ -0,0 +1,624 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SupervisorPageInfo implements org.apache.thrift.TBase<SupervisorPageInfo, SupervisorPageInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorPageInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorPageInfo");
+
+  private static final org.apache.thrift.protocol.TField SUPERVISOR_SUMMARIES_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_summaries", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField WORKER_SUMMARIES_FIELD_DESC = new org.apache.thrift.protocol.TField("worker_summaries", org.apache.thrift.protocol.TType.LIST, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SupervisorPageInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SupervisorPageInfoTupleSchemeFactory());
+  }
+
+  private List<SupervisorSummary> supervisor_summaries; // optional
+  private List<WorkerSummary> worker_summaries; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SUPERVISOR_SUMMARIES((short)1, "supervisor_summaries"),
+    WORKER_SUMMARIES((short)2, "worker_summaries");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SUPERVISOR_SUMMARIES
+          return SUPERVISOR_SUMMARIES;
+        case 2: // WORKER_SUMMARIES
+          return WORKER_SUMMARIES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.SUPERVISOR_SUMMARIES,_Fields.WORKER_SUMMARIES};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SUPERVISOR_SUMMARIES, new org.apache.thrift.meta_data.FieldMetaData("supervisor_summaries", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorSummary.class))));
+    tmpMap.put(_Fields.WORKER_SUMMARIES, new org.apache.thrift.meta_data.FieldMetaData("worker_summaries", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerSummary.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorPageInfo.class, metaDataMap);
+  }
+
+  public SupervisorPageInfo() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SupervisorPageInfo(SupervisorPageInfo other) {
+    if (other.is_set_supervisor_summaries()) {
+      List<SupervisorSummary> __this__supervisor_summaries = new ArrayList<SupervisorSummary>(other.supervisor_summaries.size());
+      for (SupervisorSummary other_element : other.supervisor_summaries) {
+        __this__supervisor_summaries.add(new SupervisorSummary(other_element));
+      }
+      this.supervisor_summaries = __this__supervisor_summaries;
+    }
+    if (other.is_set_worker_summaries()) {
+      List<WorkerSummary> __this__worker_summaries = new ArrayList<WorkerSummary>(other.worker_summaries.size());
+      for (WorkerSummary other_element : other.worker_summaries) {
+        __this__worker_summaries.add(new WorkerSummary(other_element));
+      }
+      this.worker_summaries = __this__worker_summaries;
+    }
+  }
+
+  public SupervisorPageInfo deepCopy() {
+    return new SupervisorPageInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.supervisor_summaries = null;
+    this.worker_summaries = null;
+  }
+
+  public int get_supervisor_summaries_size() {
+    return (this.supervisor_summaries == null) ? 0 : this.supervisor_summaries.size();
+  }
+
+  public java.util.Iterator<SupervisorSummary> get_supervisor_summaries_iterator() {
+    return (this.supervisor_summaries == null) ? null : this.supervisor_summaries.iterator();
+  }
+
+  public void add_to_supervisor_summaries(SupervisorSummary elem) {
+    if (this.supervisor_summaries == null) {
+      this.supervisor_summaries = new ArrayList<SupervisorSummary>();
+    }
+    this.supervisor_summaries.add(elem);
+  }
+
+  public List<SupervisorSummary> get_supervisor_summaries() {
+    return this.supervisor_summaries;
+  }
+
+  public void set_supervisor_summaries(List<SupervisorSummary> supervisor_summaries) {
+    this.supervisor_summaries = supervisor_summaries;
+  }
+
+  public void unset_supervisor_summaries() {
+    this.supervisor_summaries = null;
+  }
+
+  /** Returns true if field supervisor_summaries is set (has been assigned a value) and false otherwise */
+  public boolean is_set_supervisor_summaries() {
+    return this.supervisor_summaries != null;
+  }
+
+  public void set_supervisor_summaries_isSet(boolean value) {
+    if (!value) {
+      this.supervisor_summaries = null;
+    }
+  }
+
+  public int get_worker_summaries_size() {
+    return (this.worker_summaries == null) ? 0 : this.worker_summaries.size();
+  }
+
+  public java.util.Iterator<WorkerSummary> get_worker_summaries_iterator() {
+    return (this.worker_summaries == null) ? null : this.worker_summaries.iterator();
+  }
+
+  public void add_to_worker_summaries(WorkerSummary elem) {
+    if (this.worker_summaries == null) {
+      this.worker_summaries = new ArrayList<WorkerSummary>();
+    }
+    this.worker_summaries.add(elem);
+  }
+
+  public List<WorkerSummary> get_worker_summaries() {
+    return this.worker_summaries;
+  }
+
+  public void set_worker_summaries(List<WorkerSummary> worker_summaries) {
+    this.worker_summaries = worker_summaries;
+  }
+
+  public void unset_worker_summaries() {
+    this.worker_summaries = null;
+  }
+
+  /** Returns true if field worker_summaries is set (has been assigned a value) and false otherwise */
+  public boolean is_set_worker_summaries() {
+    return this.worker_summaries != null;
+  }
+
+  public void set_worker_summaries_isSet(boolean value) {
+    if (!value) {
+      this.worker_summaries = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SUPERVISOR_SUMMARIES:
+      if (value == null) {
+        unset_supervisor_summaries();
+      } else {
+        set_supervisor_summaries((List<SupervisorSummary>)value);
+      }
+      break;
+
+    case WORKER_SUMMARIES:
+      if (value == null) {
+        unset_worker_summaries();
+      } else {
+        set_worker_summaries((List<WorkerSummary>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SUPERVISOR_SUMMARIES:
+      return get_supervisor_summaries();
+
+    case WORKER_SUMMARIES:
+      return get_worker_summaries();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SUPERVISOR_SUMMARIES:
+      return is_set_supervisor_summaries();
+    case WORKER_SUMMARIES:
+      return is_set_worker_summaries();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SupervisorPageInfo)
+      return this.equals((SupervisorPageInfo)that);
+    return false;
+  }
+
+  public boolean equals(SupervisorPageInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_supervisor_summaries = true && this.is_set_supervisor_summaries();
+    boolean that_present_supervisor_summaries = true && that.is_set_supervisor_summaries();
+    if (this_present_supervisor_summaries || that_present_supervisor_summaries) {
+      if (!(this_present_supervisor_summaries && that_present_supervisor_summaries))
+        return false;
+      if (!this.supervisor_summaries.equals(that.supervisor_summaries))
+        return false;
+    }
+
+    boolean this_present_worker_summaries = true && this.is_set_worker_summaries();
+    boolean that_present_worker_summaries = true && that.is_set_worker_summaries();
+    if (this_present_worker_summaries || that_present_worker_summaries) {
+      if (!(this_present_worker_summaries && that_present_worker_summaries))
+        return false;
+      if (!this.worker_summaries.equals(that.worker_summaries))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_supervisor_summaries = true && (is_set_supervisor_summaries());
+    list.add(present_supervisor_summaries);
+    if (present_supervisor_summaries)
+      list.add(supervisor_summaries);
+
+    boolean present_worker_summaries = true && (is_set_worker_summaries());
+    list.add(present_worker_summaries);
+    if (present_worker_summaries)
+      list.add(worker_summaries);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SupervisorPageInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_supervisor_summaries()).compareTo(other.is_set_supervisor_summaries());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_supervisor_summaries()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_summaries, other.supervisor_summaries);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_worker_summaries()).compareTo(other.is_set_worker_summaries());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_worker_summaries()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.worker_summaries, other.worker_summaries);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SupervisorPageInfo(");
+    boolean first = true;
+
+    if (is_set_supervisor_summaries()) {
+      sb.append("supervisor_summaries:");
+      if (this.supervisor_summaries == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.supervisor_summaries);
+      }
+      first = false;
+    }
+    if (is_set_worker_summaries()) {
+      if (!first) sb.append(", ");
+      sb.append("worker_summaries:");
+      if (this.worker_summaries == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.worker_summaries);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SupervisorPageInfoStandardSchemeFactory implements SchemeFactory {
+    public SupervisorPageInfoStandardScheme getScheme() {
+      return new SupervisorPageInfoStandardScheme();
+    }
+  }
+
+  private static class SupervisorPageInfoStandardScheme extends StandardScheme<SupervisorPageInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorPageInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SUPERVISOR_SUMMARIES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list438 = iprot.readListBegin();
+                struct.supervisor_summaries = new ArrayList<SupervisorSummary>(_list438.size);
+                SupervisorSummary _elem439;
+                for (int _i440 = 0; _i440 < _list438.size; ++_i440)
+                {
+                  _elem439 = new SupervisorSummary();
+                  _elem439.read(iprot);
+                  struct.supervisor_summaries.add(_elem439);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_supervisor_summaries_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // WORKER_SUMMARIES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list441 = iprot.readListBegin();
+                struct.worker_summaries = new ArrayList<WorkerSummary>(_list441.size);
+                WorkerSummary _elem442;
+                for (int _i443 = 0; _i443 < _list441.size; ++_i443)
+                {
+                  _elem442 = new WorkerSummary();
+                  _elem442.read(iprot);
+                  struct.worker_summaries.add(_elem442);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_worker_summaries_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorPageInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.supervisor_summaries != null) {
+        if (struct.is_set_supervisor_summaries()) {
+          oprot.writeFieldBegin(SUPERVISOR_SUMMARIES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.supervisor_summaries.size()));
+            for (SupervisorSummary _iter444 : struct.supervisor_summaries)
+            {
+              _iter444.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.worker_summaries != null) {
+        if (struct.is_set_worker_summaries()) {
+          oprot.writeFieldBegin(WORKER_SUMMARIES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.worker_summaries.size()));
+            for (WorkerSummary _iter445 : struct.worker_summaries)
+            {
+              _iter445.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SupervisorPageInfoTupleSchemeFactory implements SchemeFactory {
+    public SupervisorPageInfoTupleScheme getScheme() {
+      return new SupervisorPageInfoTupleScheme();
+    }
+  }
+
+  private static class SupervisorPageInfoTupleScheme extends TupleScheme<SupervisorPageInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_supervisor_summaries()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_worker_summaries()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_supervisor_summaries()) {
+        {
+          oprot.writeI32(struct.supervisor_summaries.size());
+          for (SupervisorSummary _iter446 : struct.supervisor_summaries)
+          {
+            _iter446.write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_worker_summaries()) {
+        {
+          oprot.writeI32(struct.worker_summaries.size());
+          for (WorkerSummary _iter447 : struct.worker_summaries)
+          {
+            _iter447.write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list448 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.supervisor_summaries = new ArrayList<SupervisorSummary>(_list448.size);
+          SupervisorSummary _elem449;
+          for (int _i450 = 0; _i450 < _list448.size; ++_i450)
+          {
+            _elem449 = new SupervisorSummary();
+            _elem449.read(iprot);
+            struct.supervisor_summaries.add(_elem449);
+          }
+        }
+        struct.set_supervisor_summaries_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list451 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.worker_summaries = new ArrayList<WorkerSummary>(_list451.size);
+          WorkerSummary _elem452;
+          for (int _i453 = 0; _i453 < _list451.size; ++_i453)
+          {
+            _elem452 = new WorkerSummary();
+            _elem452.read(iprot);
+            struct.worker_summaries.add(_elem452);
+          }
+        }
+        struct.set_worker_summaries_isSet(true);
+      }
+    }
+  }
+
+}
+


[41/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
new file mode 100644
index 0000000..372df1f
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
@@ -0,0 +1,594 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.Thrift;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.ComponentCommon;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.InvalidTopologyException;
+import org.apache.storm.generated.NodeInfo;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StateSpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.metric.EventLoggerBolt;
+import org.apache.storm.metric.MetricsConsumerBolt;
+import org.apache.storm.metric.SystemBolt;
+import org.apache.storm.metric.filter.FilterByMetricName;
+import org.apache.storm.metric.util.DataPointExpander;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.task.IBolt;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.ThriftTopologyUtils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+public class StormCommon {
+    // A singleton instance allows us to mock delegated static methods in our
+    // tests by subclassing.
+    private static StormCommon _instance = new StormCommon();
+
+    /**
+     * Provide an instance of this class for delegates to use.  To mock out
+     * delegated methods, provide an instance of a subclass that overrides the
+     * implementation of the delegated method.
+     *
+     * @param common a StormCommon instance
+     * @return the previously set instance
+     */
+    public static StormCommon setInstance(StormCommon common) {
+        StormCommon oldInstance = _instance;
+        _instance = common;
+        return oldInstance;
+    }
+
+    private static final Logger LOG = LoggerFactory.getLogger(StormCommon.class);
+
+    public static final String SYSTEM_STREAM_ID = "__system";
+
+    public static final String EVENTLOGGER_COMPONENT_ID = "__eventlogger";
+    public static final String EVENTLOGGER_STREAM_ID = "__eventlog";
+
+    public static final String TOPOLOGY_METRICS_CONSUMER_CLASS = "class";
+    public static final String TOPOLOGY_METRICS_CONSUMER_ARGUMENT = "argument";
+    public static final String TOPOLOGY_METRICS_CONSUMER_MAX_RETAIN_METRIC_TUPLES = "max.retain.metric.tuples";
+    public static final String TOPOLOGY_METRICS_CONSUMER_PARALLELISM_HINT = "parallelism.hint";
+    public static final String TOPOLOGY_METRICS_CONSUMER_WHITELIST = "whitelist";
+    public static final String TOPOLOGY_METRICS_CONSUMER_BLACKLIST = "blacklist";
+    public static final String TOPOLOGY_METRICS_CONSUMER_EXPAND_MAP_TYPE = "expandMapType";
+    public static final String TOPOLOGY_METRICS_CONSUMER_METRIC_NAME_SEPARATOR = "metricNameSeparator";
+
+    @Deprecated
+    public static String getStormId(final IStormClusterState stormClusterState, final String topologyName) {
+        return stormClusterState.getTopoId(topologyName).get();
+    }
+
+    public static void validateDistributedMode(Map conf) {
+        if (ConfigUtils.isLocalMode(conf)) {
+            throw new IllegalArgumentException("Cannot start server in local mode!");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void validateIds(StormTopology topology) throws InvalidTopologyException {
+        List<String> componentIds = new ArrayList<>();
+
+        for (StormTopology._Fields field : Thrift.getTopologyFields()) {
+            if (!ThriftTopologyUtils.isWorkerHook(field) && !ThriftTopologyUtils.isDependencies(field)) {
+                Object value = topology.getFieldValue(field);
+                Map<String, Object> componentMap = (Map<String, Object>) value;
+                componentIds.addAll(componentMap.keySet());
+
+                for (String id : componentMap.keySet()) {
+                    if (Utils.isSystemId(id)) {
+                        throw new InvalidTopologyException(id + " is not a valid component id.");
+                    }
+                }
+                for (Object componentObj : componentMap.values()) {
+                    ComponentCommon common = getComponentCommon(componentObj);
+                    Set<String> streamIds = common.get_streams().keySet();
+                    for (String id : streamIds) {
+                        if (Utils.isSystemId(id)) {
+                            throw new InvalidTopologyException(id + " is not a valid stream id.");
+                        }
+                    }
+                }
+            }
+        }
+
+        List<String> offending = Utils.getRepeat(componentIds);
+        if (!offending.isEmpty()) {
+            throw new InvalidTopologyException("Duplicate component ids: " + offending);
+        }
+    }
+
+    private static boolean isEmptyInputs(ComponentCommon common) {
+        if (common.get_inputs() == null) {
+            return true;
+        } else {
+            return common.get_inputs().isEmpty();
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static Map<String, Object> allComponents(StormTopology topology) {
+        Map<String, Object> components = new HashMap<>();
+        List<StormTopology._Fields> topologyFields = Arrays.asList(Thrift.getTopologyFields());
+        for (StormTopology._Fields field : topologyFields) {
+            if (!ThriftTopologyUtils.isWorkerHook(field) && !ThriftTopologyUtils.isDependencies(field)) {
+                components.putAll(((Map) topology.getFieldValue(field)));
+            }
+        }
+        return components;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static Map<String, Object> componentConf(Object component) {
+        try {
+            Map<String, Object> conf = new HashMap<>();
+            ComponentCommon common = getComponentCommon(component);
+            String jconf = common.get_json_conf();
+            if (jconf != null) {
+                conf.putAll((Map<String, Object>) JSONValue.parseWithException(jconf));
+            }
+            return conf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void validateBasic(StormTopology topology) throws InvalidTopologyException {
+        validateIds(topology);
+
+        for (StormTopology._Fields field : Thrift.getSpoutFields()) {
+            Map<String, Object> spoutComponents = (Map<String, Object>) topology.getFieldValue(field);
+            if (spoutComponents != null) {
+                for (Object obj : spoutComponents.values()) {
+                    ComponentCommon common = getComponentCommon(obj);
+                    if (!isEmptyInputs(common)) {
+                        throw new InvalidTopologyException("May not declare inputs for a spout");
+                    }
+                }
+            }
+        }
+
+        Map<String, Object> componentMap = allComponents(topology);
+        for (Object componentObj : componentMap.values()) {
+            Map conf = componentConf(componentObj);
+            ComponentCommon common = getComponentCommon(componentObj);
+            int parallelismHintNum = Thrift.getParallelismHint(common);
+            Integer taskNum = ObjectReader.getInt(conf.get(Config.TOPOLOGY_TASKS), 0);
+            if (taskNum > 0 && parallelismHintNum <= 0) {
+                throw new InvalidTopologyException("Number of executors must be greater than 0 when number of tasks is greater than 0");
+            }
+        }
+    }
+
+    private static Set<String> getStreamOutputFields(Map<String, StreamInfo> streams) {
+        Set<String> outputFields = new HashSet<>();
+        for (StreamInfo streamInfo : streams.values()) {
+            outputFields.addAll(streamInfo.get_output_fields());
+        }
+        return outputFields;
+    }
+
+    public static void validateStructure(StormTopology topology) throws InvalidTopologyException {
+        Map<String, Object> componentMap = allComponents(topology);
+        for (Map.Entry<String, Object> entry : componentMap.entrySet()) {
+            String componentId = entry.getKey();
+            ComponentCommon common = getComponentCommon(entry.getValue());
+            Map<GlobalStreamId, Grouping> inputs = common.get_inputs();
+            for (Map.Entry<GlobalStreamId, Grouping> input : inputs.entrySet()) {
+                String sourceStreamId = input.getKey().get_streamId();
+                String sourceComponentId = input.getKey().get_componentId();
+                if (!componentMap.keySet().contains(sourceComponentId)) {
+                    throw new InvalidTopologyException("Component: [" + componentId +
+                            "] subscribes from non-existent component [" + sourceComponentId + "]");
+                }
+
+                ComponentCommon sourceComponent = getComponentCommon(componentMap.get(sourceComponentId));
+                if (!sourceComponent.get_streams().containsKey(sourceStreamId)) {
+                    throw new InvalidTopologyException("Component: [" + componentId +
+                            "] subscribes from non-existent stream: " +
+                            "[" + sourceStreamId + "] of component [" + sourceComponentId + "]");
+                }
+
+                Grouping grouping = input.getValue();
+                if (Thrift.groupingType(grouping) == Grouping._Fields.FIELDS) {
+                    List<String> fields = new ArrayList<>(grouping.get_fields());
+                    Map<String, StreamInfo> streams = sourceComponent.get_streams();
+                    Set<String> sourceOutputFields = getStreamOutputFields(streams);
+                    fields.removeAll(sourceOutputFields);
+                    if (fields.size() != 0) {
+                        throw new InvalidTopologyException("Component: [" + componentId +
+                                "] subscribes from stream: [" + sourceStreamId + "] of component " +
+                                "[" + sourceComponentId + "] + with non-existent fields: " + fields);
+                    }
+                }
+            }
+        }
+    }
+
+    public static Map<GlobalStreamId, Grouping> ackerInputs(StormTopology topology) {
+        Map<GlobalStreamId, Grouping> inputs = new HashMap<>();
+        Set<String> boltIds = topology.get_bolts().keySet();
+        Set<String> spoutIds = topology.get_spouts().keySet();
+
+        for (String id : spoutIds) {
+            inputs.put(Utils.getGlobalStreamId(id, Acker.ACKER_INIT_STREAM_ID),
+                    Thrift.prepareFieldsGrouping(Arrays.asList("id")));
+        }
+
+        for (String id : boltIds) {
+            inputs.put(Utils.getGlobalStreamId(id, Acker.ACKER_ACK_STREAM_ID),
+                    Thrift.prepareFieldsGrouping(Arrays.asList("id")));
+            inputs.put(Utils.getGlobalStreamId(id, Acker.ACKER_FAIL_STREAM_ID),
+                    Thrift.prepareFieldsGrouping(Arrays.asList("id")));
+            inputs.put(Utils.getGlobalStreamId(id, Acker.ACKER_RESET_TIMEOUT_STREAM_ID),
+                    Thrift.prepareFieldsGrouping(Arrays.asList("id")));
+        }
+        return inputs;
+    }
+
+    public static IBolt makeAckerBolt() {
+        return _instance.makeAckerBoltImpl();
+    }
+
+    public IBolt makeAckerBoltImpl() {
+        return new Acker();
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void addAcker(Map conf, StormTopology topology) {
+        int ackerNum = ObjectReader.getInt(conf.get(Config.TOPOLOGY_ACKER_EXECUTORS), ObjectReader.getInt(conf.get(Config.TOPOLOGY_WORKERS)));
+        Map<GlobalStreamId, Grouping> inputs = ackerInputs(topology);
+
+        Map<String, StreamInfo> outputStreams = new HashMap<String, StreamInfo>();
+        outputStreams.put(Acker.ACKER_ACK_STREAM_ID, Thrift.directOutputFields(Arrays.asList("id", "time-delta-ms")));
+        outputStreams.put(Acker.ACKER_FAIL_STREAM_ID, Thrift.directOutputFields(Arrays.asList("id", "time-delta-ms")));
+        outputStreams.put(Acker.ACKER_RESET_TIMEOUT_STREAM_ID, Thrift.directOutputFields(Arrays.asList("id", "time-delta-ms")));
+
+        Map<String, Object> ackerConf = new HashMap<>();
+        ackerConf.put(Config.TOPOLOGY_TASKS, ackerNum);
+        ackerConf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, ObjectReader.getInt(conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)));
+
+        Bolt acker = Thrift.prepareSerializedBoltDetails(inputs, makeAckerBolt(), outputStreams, ackerNum, ackerConf);
+
+        for (Bolt bolt : topology.get_bolts().values()) {
+            ComponentCommon common = bolt.get_common();
+            common.put_to_streams(Acker.ACKER_ACK_STREAM_ID, Thrift.outputFields(Arrays.asList("id", "ack-val")));
+            common.put_to_streams(Acker.ACKER_FAIL_STREAM_ID, Thrift.outputFields(Arrays.asList("id")));
+            common.put_to_streams(Acker.ACKER_RESET_TIMEOUT_STREAM_ID, Thrift.outputFields(Arrays.asList("id")));
+        }
+
+        for (SpoutSpec spout : topology.get_spouts().values()) {
+            ComponentCommon common = spout.get_common();
+            Map spoutConf = componentConf(spout);
+            spoutConf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS,
+                    ObjectReader.getInt(conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)));
+            common.set_json_conf(JSONValue.toJSONString(spoutConf));
+            common.put_to_streams(Acker.ACKER_INIT_STREAM_ID,
+                    Thrift.outputFields(Arrays.asList("id", "init-val", "spout-task")));
+            common.put_to_inputs(Utils.getGlobalStreamId(Acker.ACKER_COMPONENT_ID, Acker.ACKER_ACK_STREAM_ID),
+                    Thrift.prepareDirectGrouping());
+            common.put_to_inputs(Utils.getGlobalStreamId(Acker.ACKER_COMPONENT_ID, Acker.ACKER_FAIL_STREAM_ID),
+                    Thrift.prepareDirectGrouping());
+            common.put_to_inputs(Utils.getGlobalStreamId(Acker.ACKER_COMPONENT_ID, Acker.ACKER_RESET_TIMEOUT_STREAM_ID),
+                    Thrift.prepareDirectGrouping());
+        }
+
+        topology.put_to_bolts(Acker.ACKER_COMPONENT_ID, acker);
+    }
+
+    public static ComponentCommon getComponentCommon(Object component) {
+        ComponentCommon common = null;
+        if (component instanceof StateSpoutSpec) {
+            common = ((StateSpoutSpec) component).get_common();
+        } else if (component instanceof SpoutSpec) {
+            common = ((SpoutSpec) component).get_common();
+        } else if (component instanceof Bolt) {
+            common = ((Bolt) component).get_common();
+        }
+        return common;
+    }
+
+    public static void addMetricStreams(StormTopology topology) {
+        for (Object component : allComponents(topology).values()) {
+            ComponentCommon common = getComponentCommon(component);
+            StreamInfo streamInfo = Thrift.outputFields(Arrays.asList("task-info", "data-points"));
+            common.put_to_streams(Constants.METRICS_STREAM_ID, streamInfo);
+        }
+    }
+
+    public static void addSystemStreams(StormTopology topology) {
+        for (Object component : allComponents(topology).values()) {
+            ComponentCommon common = getComponentCommon(component);
+            StreamInfo streamInfo = Thrift.outputFields(Arrays.asList("event"));
+            common.put_to_streams(SYSTEM_STREAM_ID, streamInfo);
+        }
+    }
+
+    public static List<String> eventLoggerBoltFields() {
+        return Arrays.asList(EventLoggerBolt.FIELD_COMPONENT_ID, EventLoggerBolt.FIELD_MESSAGE_ID,
+                EventLoggerBolt.FIELD_TS, EventLoggerBolt.FIELD_VALUES);
+    }
+
+    public static Map<GlobalStreamId, Grouping> eventLoggerInputs(StormTopology topology) {
+        Map<GlobalStreamId, Grouping> inputs = new HashMap<GlobalStreamId, Grouping>();
+        Set<String> allIds = new HashSet<String>();
+        allIds.addAll(topology.get_bolts().keySet());
+        allIds.addAll(topology.get_spouts().keySet());
+
+        for (String id : allIds) {
+            inputs.put(Utils.getGlobalStreamId(id, EVENTLOGGER_STREAM_ID),
+                    Thrift.prepareFieldsGrouping(Arrays.asList("component-id")));
+        }
+        return inputs;
+    }
+
+    public static void addEventLogger(Map conf, StormTopology topology) {
+        Integer numExecutors = ObjectReader.getInt(conf.get(Config.TOPOLOGY_EVENTLOGGER_EXECUTORS),
+                ObjectReader.getInt(conf.get(Config.TOPOLOGY_WORKERS)));
+        HashMap<String, Object> componentConf = new HashMap<>();
+        componentConf.put(Config.TOPOLOGY_TASKS, numExecutors);
+        componentConf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, ObjectReader.getInt(conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)));
+        Bolt eventLoggerBolt = Thrift.prepareSerializedBoltDetails(
+                eventLoggerInputs(topology), new EventLoggerBolt(), null, numExecutors, componentConf);
+
+        for (Object component : allComponents(topology).values()) {
+            ComponentCommon common = getComponentCommon(component);
+            common.put_to_streams(EVENTLOGGER_STREAM_ID, Thrift.outputFields(eventLoggerBoltFields()));
+        }
+        topology.put_to_bolts(EVENTLOGGER_COMPONENT_ID, eventLoggerBolt);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static Map<String, Bolt> metricsConsumerBoltSpecs(Map conf, StormTopology topology) {
+        Map<String, Bolt> metricsConsumerBolts = new HashMap<>();
+
+        Set<String> componentIdsEmitMetrics = new HashSet<>();
+        componentIdsEmitMetrics.addAll(allComponents(topology).keySet());
+        componentIdsEmitMetrics.add(Constants.SYSTEM_COMPONENT_ID);
+
+        Map<GlobalStreamId, Grouping> inputs = new HashMap<>();
+        for (String componentId : componentIdsEmitMetrics) {
+            inputs.put(Utils.getGlobalStreamId(componentId, Constants.METRICS_STREAM_ID), Thrift.prepareShuffleGrouping());
+        }
+
+        List<Map<String, Object>> registerInfo = (List<Map<String, Object>>) conf.get(Config.TOPOLOGY_METRICS_CONSUMER_REGISTER);
+        if (registerInfo != null) {
+            Map<String, Integer> classOccurrencesMap = new HashMap<String, Integer>();
+            for (Map<String, Object> info : registerInfo) {
+                String className = (String) info.get(TOPOLOGY_METRICS_CONSUMER_CLASS);
+                Object argument = info.get(TOPOLOGY_METRICS_CONSUMER_ARGUMENT);
+                Integer maxRetainMetricTuples = ObjectReader.getInt(info.get(
+                    TOPOLOGY_METRICS_CONSUMER_MAX_RETAIN_METRIC_TUPLES), 100);
+                Integer phintNum = ObjectReader.getInt(info.get(TOPOLOGY_METRICS_CONSUMER_PARALLELISM_HINT), 1);
+                Map<String, Object> metricsConsumerConf = new HashMap<String, Object>();
+                metricsConsumerConf.put(Config.TOPOLOGY_TASKS, phintNum);
+                List<String> whitelist = (List<String>) info.get(
+                    TOPOLOGY_METRICS_CONSUMER_WHITELIST);
+                List<String> blacklist = (List<String>) info.get(
+                    TOPOLOGY_METRICS_CONSUMER_BLACKLIST);
+                FilterByMetricName filterPredicate = new FilterByMetricName(whitelist, blacklist);
+                Boolean expandMapType = ObjectReader.getBoolean(info.get(
+                    TOPOLOGY_METRICS_CONSUMER_EXPAND_MAP_TYPE), false);
+                String metricNameSeparator = ObjectReader.getString(info.get(
+                    TOPOLOGY_METRICS_CONSUMER_METRIC_NAME_SEPARATOR), ".");
+                DataPointExpander expander = new DataPointExpander(expandMapType, metricNameSeparator);
+                MetricsConsumerBolt boltInstance = new MetricsConsumerBolt(className, argument,
+                    maxRetainMetricTuples, filterPredicate, expander);
+                Bolt metricsConsumerBolt = Thrift.prepareSerializedBoltDetails(inputs,
+                    boltInstance, null, phintNum, metricsConsumerConf);
+
+                String id = className;
+                if (classOccurrencesMap.containsKey(className)) {
+                    // e.g. [\"a\", \"b\", \"a\"]) => [\"a\", \"b\", \"a#2\"]"
+                    int occurrenceNum = classOccurrencesMap.get(className);
+                    occurrenceNum++;
+                    classOccurrencesMap.put(className, occurrenceNum);
+                    id = Constants.METRICS_COMPONENT_ID_PREFIX + className + "#" + occurrenceNum;
+                } else {
+                    classOccurrencesMap.put(className, 1);
+                }
+                metricsConsumerBolts.put(id, metricsConsumerBolt);
+            }
+        }
+        return metricsConsumerBolts;
+    }
+
+    public static void addMetricComponents(Map conf, StormTopology topology) {
+        Map<String, Bolt> metricsConsumerBolts = metricsConsumerBoltSpecs(conf, topology);
+        for (Map.Entry<String, Bolt> entry : metricsConsumerBolts.entrySet()) {
+            topology.put_to_bolts(entry.getKey(), entry.getValue());
+        }
+    }
+
+    @SuppressWarnings("unused")
+    public static void addSystemComponents(Map conf, StormTopology topology) {
+        Map<String, StreamInfo> outputStreams = new HashMap<>();
+        outputStreams.put(Constants.SYSTEM_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("rate_secs")));
+        outputStreams.put(Constants.METRICS_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("interval")));
+        outputStreams.put(Constants.CREDENTIALS_CHANGED_STREAM_ID, Thrift.outputFields(Arrays.asList("creds")));
+
+        Map<String, Object> boltConf = new HashMap<>();
+        boltConf.put(Config.TOPOLOGY_TASKS, 0);
+
+        Bolt systemBoltSpec = Thrift.prepareSerializedBoltDetails(null, new SystemBolt(), outputStreams, 0, boltConf);
+        topology.put_to_bolts(Constants.SYSTEM_COMPONENT_ID, systemBoltSpec);
+    }
+
+    public static StormTopology systemTopology(Map stormConf, StormTopology topology) throws InvalidTopologyException {
+        return _instance.systemTopologyImpl(stormConf, topology);
+    }
+
+    protected StormTopology systemTopologyImpl(Map stormConf, StormTopology topology) throws InvalidTopologyException {
+        validateBasic(topology);
+
+        StormTopology ret = topology.deepCopy();
+        addAcker(stormConf, ret);
+        if (hasEventLoggers(stormConf)) {
+            addEventLogger(stormConf, ret);
+        }
+        addMetricComponents(stormConf, ret);
+        addSystemComponents(stormConf, ret);
+        addMetricStreams(ret);
+        addSystemStreams(ret);
+
+        validateStructure(ret);
+
+        return ret;
+    }
+
+    public static boolean hasAckers(Map stormConf) {
+        Object ackerNum = stormConf.get(Config.TOPOLOGY_ACKER_EXECUTORS);
+        return ackerNum == null || ObjectReader.getInt(ackerNum) > 0;
+    }
+
+    public static boolean hasEventLoggers(Map stormConf) {
+        Object eventLoggerNum = stormConf.get(Config.TOPOLOGY_EVENTLOGGER_EXECUTORS);
+        return eventLoggerNum == null || ObjectReader.getInt(eventLoggerNum) > 0;
+    }
+
+    public static int numStartExecutors(Object component) throws InvalidTopologyException {
+        ComponentCommon common = getComponentCommon(component);
+        return Thrift.getParallelismHint(common);
+    }
+
+    public static Map<Integer, String> stormTaskInfo(StormTopology userTopology, Map stormConf) throws InvalidTopologyException {
+        return _instance.stormTaskInfoImpl(userTopology, stormConf);
+    }
+
+    /*
+     * Returns map from task -> componentId
+     */
+    protected Map<Integer, String> stormTaskInfoImpl(StormTopology userTopology, Map stormConf) throws InvalidTopologyException {
+        Map<Integer, String> taskIdToComponentId = new HashMap<>();
+
+        StormTopology systemTopology = systemTopology(stormConf, userTopology);
+        Map<String, Object> components = allComponents(systemTopology);
+        Map<String, Integer> componentIdToTaskNum = new TreeMap<>();
+        for (Map.Entry<String, Object> entry : components.entrySet()) {
+            Map conf = componentConf(entry.getValue());
+            Object taskNum = conf.get(Config.TOPOLOGY_TASKS);
+            componentIdToTaskNum.put(entry.getKey(), ObjectReader.getInt(taskNum));
+        }
+
+        int taskId = 1;
+        for (Map.Entry<String, Integer> entry : componentIdToTaskNum.entrySet()) {
+            String componentId = entry.getKey();
+            Integer taskNum = entry.getValue();
+            while (taskNum > 0) {
+                taskIdToComponentId.put(taskId, componentId);
+                taskNum--;
+                taskId++;
+            }
+        }
+        return taskIdToComponentId;
+    }
+
+    public static List<Integer> executorIdToTasks(List<Long> executorId) {
+        List<Integer> taskIds = new ArrayList<>();
+        int taskId = executorId.get(0).intValue();
+        while (taskId <= executorId.get(1).intValue()) {
+            taskIds.add(taskId);
+            taskId++;
+        }
+        return taskIds;
+    }
+
+    public static Map<Integer, NodeInfo> taskToNodeport(Map<List<Long>, NodeInfo> executorToNodePort) {
+        Map<Integer, NodeInfo> tasksToNodePort = new HashMap<>();
+        for (Map.Entry<List<Long>, NodeInfo> entry : executorToNodePort.entrySet()) {
+            List<Integer> taskIds = executorIdToTasks(entry.getKey());
+            for (Integer taskId : taskIds) {
+                tasksToNodePort.put(taskId, entry.getValue());
+            }
+        }
+        return tasksToNodePort;
+    }
+
+    public static IAuthorizer mkAuthorizationHandler(String klassName, Map conf)
+            throws IllegalAccessException, InstantiationException, ClassNotFoundException {
+        return _instance.mkAuthorizationHandlerImpl(klassName, conf);
+    }
+
+    protected IAuthorizer mkAuthorizationHandlerImpl(String klassName, Map conf)
+            throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+        IAuthorizer aznHandler = null;
+        if (StringUtils.isNotBlank(klassName)) {
+            Class<?> aznClass = Class.forName(klassName);
+            if (aznClass != null) {
+                aznHandler = (IAuthorizer) aznClass.newInstance();
+                if (aznHandler != null) {
+                    aznHandler.prepare(conf);
+                }
+                LOG.debug("authorization class name:{}, class:{}, handler:{}", klassName, aznClass, aznHandler);
+            }
+        }
+
+        return aznHandler;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static WorkerTopologyContext makeWorkerContext(Map<String, Object> workerData) {
+        try {
+            StormTopology stormTopology = (StormTopology) workerData.get(Constants.SYSTEM_TOPOLOGY);
+            Map stormConf = (Map) workerData.get(Constants.STORM_CONF);
+            Map<Integer, String> taskToComponent = (Map<Integer, String>) workerData.get(Constants.TASK_TO_COMPONENT);
+            Map<String, List<Integer>> componentToSortedTasks =
+                    (Map<String, List<Integer>>) workerData.get(Constants.COMPONENT_TO_SORTED_TASKS);
+            Map<String, Map<String, Fields>> componentToStreamToFields =
+                    (Map<String, Map<String, Fields>>) workerData.get(Constants.COMPONENT_TO_STREAM_TO_FIELDS);
+            String stormId = (String) workerData.get(Constants.STORM_ID);
+            Map conf = (Map) workerData.get(Constants.CONF);
+            Integer port = (Integer) workerData.get(Constants.PORT);
+            String codeDir = ConfigUtils.supervisorStormResourcesPath(ConfigUtils.supervisorStormDistRoot(conf, stormId));
+            String pidDir = ConfigUtils.workerPidsRoot(conf, stormId);
+            List<Integer> workerTasks = (List<Integer>) workerData.get(Constants.TASK_IDS);
+            Map<String, Object> defaultResources = (Map<String, Object>) workerData.get(Constants.DEFAULT_SHARED_RESOURCES);
+            Map<String, Object> userResources = (Map<String, Object>) workerData.get(Constants.USER_SHARED_RESOURCES);
+            return new WorkerTopologyContext(stormTopology, stormConf, taskToComponent, componentToSortedTasks,
+                    componentToStreamToFields, stormId, codeDir, pidDir, port, workerTasks, defaultResources, userResources);
+        } catch (IOException e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/Task.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/Task.java b/storm-client/src/jvm/org/apache/storm/daemon/Task.java
new file mode 100644
index 0000000..6316564
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/Task.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon;
+
+import org.apache.storm.Config;
+import org.apache.storm.Thrift;
+import org.apache.storm.daemon.metrics.BuiltinMetrics;
+import org.apache.storm.daemon.metrics.BuiltinMetricsUtil;
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.ComponentObject;
+import org.apache.storm.generated.JavaObject;
+import org.apache.storm.generated.ShellComponent;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StateSpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.grouping.LoadMapping;
+import org.apache.storm.hooks.ITaskHook;
+import org.apache.storm.hooks.info.EmitInfo;
+import org.apache.storm.spout.ShellSpout;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.task.ShellBolt;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+public class Task {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Task.class);
+
+    private Executor executor;
+    private WorkerState workerData;
+    private TopologyContext systemTopologyContext;
+    private TopologyContext userTopologyContext;
+    private WorkerTopologyContext workerTopologyContext;
+    private LoadMapping loadMapping;
+    private Integer taskId;
+    private String componentId;
+    private Object taskObject; // Spout/Bolt object
+    private Map stormConf;
+    private Callable<Boolean> emitSampler;
+    private CommonStats executorStats;
+    private Map<String, Map<String, LoadAwareCustomStreamGrouping>> streamComponentToGrouper;
+    private BuiltinMetrics builtInMetrics;
+    private boolean debug;
+
+    public Task(Executor executor, Integer taskId) throws IOException {
+        this.taskId = taskId;
+        this.executor = executor;
+        this.workerData = executor.getWorkerData();
+        this.stormConf = executor.getStormConf();
+        this.componentId = executor.getComponentId();
+        this.streamComponentToGrouper = executor.getStreamToComponentToGrouper();
+        this.executorStats = executor.getStats();
+        this.builtInMetrics = BuiltinMetricsUtil.mkData(executor.getType(), this.executorStats);
+        this.workerTopologyContext = executor.getWorkerTopologyContext();
+        this.emitSampler = ConfigUtils.mkStatsSampler(stormConf);
+        this.loadMapping = workerData.getLoadMapping();
+        this.systemTopologyContext = mkTopologyContext(workerData.getSystemTopology());
+        this.userTopologyContext = mkTopologyContext(workerData.getTopology());
+        this.taskObject = mkTaskObject();
+        this.debug = stormConf.containsKey(Config.TOPOLOGY_DEBUG) && (Boolean) stormConf.get(Config.TOPOLOGY_DEBUG);
+        this.addTaskHooks();
+    }
+
+    public List<Integer> getOutgoingTasks(Integer outTaskId, String stream, List<Object> values) {
+        if (debug) {
+            LOG.info("Emitting direct: {}; {} {} {} ", outTaskId, componentId, stream, values);
+        }
+        String targetComponent = workerTopologyContext.getComponentId(outTaskId);
+        Map<String, LoadAwareCustomStreamGrouping> componentGrouping = streamComponentToGrouper.get(stream);
+        LoadAwareCustomStreamGrouping grouping = componentGrouping.get(targetComponent);
+        if (null == grouping) {
+            outTaskId = null;
+        }
+        if (grouping != null && grouping != GrouperFactory.DIRECT) {
+            throw new IllegalArgumentException("Cannot emitDirect to a task expecting a regular grouping");
+        }
+        new EmitInfo(values, stream, taskId, Collections.singletonList(outTaskId)).applyOn(userTopologyContext);
+        try {
+            if (emitSampler.call()) {
+                executorStats.emittedTuple(stream);
+                if (null != outTaskId) {
+                    executorStats.transferredTuples(stream, 1);
+                }
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        if (null != outTaskId) {
+            return Collections.singletonList(outTaskId);
+        }
+        return new ArrayList<>(0);
+    }
+
+    public List<Integer> getOutgoingTasks(String stream, List<Object> values) {
+        if (debug) {
+            LOG.info("Emitting: {} {} {}", componentId, stream, values);
+        }
+
+        List<Integer> outTasks = new ArrayList<>();
+        if (!streamComponentToGrouper.containsKey(stream)) {
+            throw new IllegalArgumentException("Unknown stream ID: " + stream);
+        }
+        if (null != streamComponentToGrouper.get(stream)) {
+            // null value for __system
+            for (LoadAwareCustomStreamGrouping grouper : streamComponentToGrouper.get(stream).values()) {
+                if (grouper == GrouperFactory.DIRECT) {
+                    throw new IllegalArgumentException("Cannot do regular emit to direct stream");
+                }
+                List<Integer> compTasks = grouper.chooseTasks(taskId, values, loadMapping);
+                outTasks.addAll(compTasks);
+            }
+        }
+        new EmitInfo(values, stream, taskId, outTasks).applyOn(userTopologyContext);
+        try {
+            if (emitSampler.call()) {
+                executorStats.emittedTuple(stream);
+                executorStats.transferredTuples(stream, outTasks.size());
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        return outTasks;
+    }
+
+    public Tuple getTuple(String stream, List values) {
+        return new TupleImpl(systemTopologyContext, values, systemTopologyContext.getThisTaskId(), stream);
+    }
+
+    public Integer getTaskId() {
+        return taskId;
+    }
+
+    public String getComponentId() {
+        return componentId;
+    }
+
+    public TopologyContext getUserContext() {
+        return userTopologyContext;
+    }
+
+    public Object getTaskObject() {
+        return taskObject;
+    }
+
+    public BuiltinMetrics getBuiltInMetrics() {
+        return builtInMetrics;
+    }
+
+    private TopologyContext mkTopologyContext(StormTopology topology) throws IOException {
+        Map conf = workerData.getConf();
+        return new TopologyContext(
+            topology,
+            workerData.getTopologyConf(),
+            workerData.getTaskToComponent(),
+            workerData.getComponentToSortedTasks(),
+            workerData.getComponentToStreamToFields(),
+            workerData.getTopologyId(),
+            ConfigUtils.supervisorStormResourcesPath(
+                    ConfigUtils.supervisorStormDistRoot(conf, workerData.getTopologyId())),
+                    ConfigUtils.workerPidsRoot(conf, workerData.getWorkerId()),
+            taskId,
+            workerData.getPort(), workerData.getTaskIds(),
+            workerData.getDefaultSharedResources(),
+            workerData.getUserSharedResources(),
+            executor.getSharedExecutorData(),
+            executor.getIntervalToTaskToMetricToRegistry(),
+            executor.getOpenOrPrepareWasCalled());
+    }
+
+    private Object mkTaskObject() {
+        StormTopology topology = systemTopologyContext.getRawTopology();
+        Map<String, SpoutSpec> spouts = topology.get_spouts();
+        Map<String, Bolt> bolts = topology.get_bolts();
+        Map<String, StateSpoutSpec> stateSpouts = topology.get_state_spouts();
+        Object result;
+        ComponentObject componentObject;
+        if (spouts.containsKey(componentId)) {
+            componentObject = spouts.get(componentId).get_spout_object();
+        } else if (bolts.containsKey(componentId)) {
+            componentObject = bolts.get(componentId).get_bolt_object();
+        } else if (stateSpouts.containsKey(componentId)) {
+            componentObject = stateSpouts.get(componentId).get_state_spout_object();
+        } else {
+            throw new RuntimeException("Could not find " + componentId + " in " + topology);
+        }
+        result = Utils.getSetComponentObject(componentObject);
+
+        if (result instanceof ShellComponent) {
+            if (spouts.containsKey(componentId)) {
+                result = new ShellSpout((ShellComponent) result);
+            } else {
+                result = new ShellBolt((ShellComponent) result);
+            }
+        }
+
+        if (result instanceof JavaObject) {
+            result = Thrift.instantiateJavaObject((JavaObject) result);
+        }
+
+        return result;
+    }
+
+    private void addTaskHooks() {
+        List<String> hooksClassList = (List<String>) stormConf.get(Config.TOPOLOGY_AUTO_TASK_HOOKS);
+        if (null != hooksClassList) {
+            for (String hookClass : hooksClassList) {
+                try {
+                    userTopologyContext.addTaskHook(((ITaskHook) Class.forName(hookClass).newInstance()));
+                } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+                    throw new RuntimeException("Failed to add hook: " + hookClass, e);
+                }
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinBoltMetrics.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinBoltMetrics.java b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinBoltMetrics.java
new file mode 100644
index 0000000..b9677f6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinBoltMetrics.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon.metrics;
+
+import org.apache.storm.metric.internal.MultiCountStatAndMetric;
+import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
+import org.apache.storm.stats.BoltExecutorStats;
+
+public class BuiltinBoltMetrics extends BuiltinMetrics {
+    private final MultiCountStatAndMetric ackCount;
+    private final MultiCountStatAndMetric failCount;
+    private final MultiCountStatAndMetric emitCount;
+    private final MultiCountStatAndMetric executeCount;
+    private final MultiCountStatAndMetric transferCount;
+    private final MultiLatencyStatAndMetric executeLatency;
+    private final MultiLatencyStatAndMetric processLatency;
+
+    public BuiltinBoltMetrics(BoltExecutorStats stats) {
+        this.ackCount = stats.getAcked();
+        this.failCount = stats.getFailed();
+        this.emitCount = stats.getEmitted();
+        this.executeCount = stats.getExecuted();
+        this.transferCount = stats.getTransferred();
+        this.executeLatency = stats.getExecuteLatencies();
+        this.processLatency = stats.getProcessLatencies();
+
+        this.metricMap.put("ack-count", ackCount);
+        this.metricMap.put("fail-count", failCount);
+        this.metricMap.put("emit-count", emitCount);
+        this.metricMap.put("transfer-count", transferCount);
+        this.metricMap.put("execute-count", executeCount);
+        this.metricMap.put("process-latency", processLatency);
+        this.metricMap.put("execute-latency", executeLatency);
+    }
+
+    public MultiCountStatAndMetric getAckCount() {
+        return ackCount;
+    }
+
+    public MultiCountStatAndMetric getFailCount() {
+        return failCount;
+    }
+
+    public MultiCountStatAndMetric getEmitCount() {
+        return emitCount;
+    }
+
+    public MultiCountStatAndMetric getTransferCount() {
+        return transferCount;
+    }
+
+    public MultiCountStatAndMetric getExecuteCount() {
+        return executeCount;
+    }
+
+    public MultiLatencyStatAndMetric getExecuteLatency() {
+        return executeLatency;
+    }
+
+    public MultiLatencyStatAndMetric getProcessLatency() {
+        return processLatency;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetrics.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetrics.java b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetrics.java
new file mode 100644
index 0000000..0dacad1
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetrics.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon.metrics;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.task.TopologyContext;
+
+public abstract class BuiltinMetrics {
+    protected final Map<String, IMetric> metricMap = new HashMap<>();
+
+    public void registerAll(Map stormConf, TopologyContext context) {
+        for (Map.Entry<String, IMetric> entry : metricMap.entrySet()) {
+            BuiltinMetricsUtil.registerMetric("__" + entry.getKey(), entry.getValue(), stormConf, context);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java
new file mode 100644
index 0000000..2827420
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon.metrics;
+
+import org.apache.storm.Config;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IStatefulObject;
+import org.apache.storm.metric.api.StateMetric;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.TopologyContext;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class BuiltinMetricsUtil {
+    public static BuiltinMetrics mkData(String type, CommonStats stats) {
+        if (StatsUtil.SPOUT.equals(type)) {
+            return new BuiltinSpoutMetrics((SpoutExecutorStats) stats);
+        } else if (StatsUtil.BOLT.equals(type)) {
+            return new BuiltinBoltMetrics((BoltExecutorStats) stats);
+        }
+        throw new RuntimeException("Invalid component type!");
+    }
+
+    public static void registerIconnectionServerMetric(Object server, Map stormConf, TopologyContext context) {
+        if (server instanceof IStatefulObject) {
+            registerMetric("__recv-iconnection", new StateMetric((IStatefulObject) server), stormConf, context);
+        }
+    }
+
+    public static void registerIconnectionClientMetrics(final Map nodePortToSocket, Map stormConf, TopologyContext context) {
+        IMetric metric = new IMetric() {
+            @Override
+            public Object getValueAndReset() {
+                Map<Object, Object> ret = new HashMap<>();
+                for (Object o : nodePortToSocket.entrySet()) {
+                    Map.Entry entry = (Map.Entry) o;
+                    Object nodePort = entry.getKey();
+                    Object connection = entry.getValue();
+                    if (connection instanceof IStatefulObject) {
+                        ret.put(nodePort, ((IStatefulObject) connection).getState());
+                    }
+                }
+                return ret;
+            }
+        };
+        registerMetric("__send-iconnection", metric, stormConf, context);
+    }
+
+    public static void registerQueueMetrics(Map queues, Map stormConf, TopologyContext context) {
+        for (Object o : queues.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            String name = "__" + entry.getKey();
+            IMetric metric = new StateMetric((IStatefulObject) entry.getValue());
+            registerMetric(name, metric, stormConf, context);
+        }
+    }
+
+    public static void registerMetric(String name, IMetric metric, Map stormConf, TopologyContext context) {
+        int bucketSize = ((Number) stormConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)).intValue();
+        context.registerMetric(name, metric, bucketSize);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinSpoutMetrics.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinSpoutMetrics.java b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinSpoutMetrics.java
new file mode 100644
index 0000000..5eef4bb
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinSpoutMetrics.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon.metrics;
+
+import org.apache.storm.metric.internal.MultiCountStatAndMetric;
+import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
+import org.apache.storm.stats.SpoutExecutorStats;
+
+public class BuiltinSpoutMetrics extends BuiltinMetrics {
+    private final MultiCountStatAndMetric ackCount;
+    private final MultiCountStatAndMetric failCount;
+    private final MultiCountStatAndMetric emitCount;
+    private final MultiCountStatAndMetric transferCount;
+    private final MultiLatencyStatAndMetric completeLatency;
+
+    public BuiltinSpoutMetrics(SpoutExecutorStats stats) {
+        this.ackCount = stats.getAcked();
+        this.failCount = stats.getFailed();
+        this.emitCount = stats.getEmitted();
+        this.transferCount = stats.getTransferred();
+        this.completeLatency = stats.getCompleteLatencies();
+
+        this.metricMap.put("ack-count", ackCount);
+        this.metricMap.put("fail-count", failCount);
+        this.metricMap.put("emit-count", emitCount);
+        this.metricMap.put("transfer-count", transferCount);
+        this.metricMap.put("complete-latency", completeLatency);
+    }
+
+    public MultiCountStatAndMetric getAckCount() {
+        return ackCount;
+    }
+
+    public MultiCountStatAndMetric getFailCount() {
+        return failCount;
+    }
+
+    public MultiCountStatAndMetric getEmitCount() {
+        return emitCount;
+    }
+
+    public MultiCountStatAndMetric getTransferCount() {
+        return transferCount;
+    }
+
+    public MultiLatencyStatAndMetric getCompleteLatency() {
+        return completeLatency;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/metrics/SpoutThrottlingMetrics.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/metrics/SpoutThrottlingMetrics.java b/storm-client/src/jvm/org/apache/storm/daemon/metrics/SpoutThrottlingMetrics.java
new file mode 100644
index 0000000..3c7f524
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/metrics/SpoutThrottlingMetrics.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon.metrics;
+
+import org.apache.storm.metric.api.CountMetric;
+import org.apache.storm.stats.CommonStats;
+
+public class SpoutThrottlingMetrics extends BuiltinMetrics {
+    private final CountMetric skippedMaxSpout = new CountMetric();
+    private final CountMetric skippedThrottle = new CountMetric();
+    private final CountMetric skippedInactive = new CountMetric();
+
+    public SpoutThrottlingMetrics() {
+        this.metricMap.put("skipped-max-spout", skippedMaxSpout);
+        this.metricMap.put("skipped-throttle", skippedThrottle);
+        this.metricMap.put("skipped-inactive", skippedInactive);
+    }
+
+    public CountMetric getSkippedMaxSpout() {
+        return skippedMaxSpout;
+    }
+
+    public CountMetric getSkippedThrottle() {
+        return skippedThrottle;
+    }
+
+    public CountMetric getSkippedInactive() {
+        return skippedInactive;
+    }
+
+    public void skippedMaxSpout(CommonStats stats) {
+        this.skippedMaxSpout.incrBy(stats.getRate());
+    }
+
+    public void skippedThrottle(CommonStats stats) {
+        this.skippedThrottle.incrBy(stats.getRate());
+    }
+
+    public void skippedInactive(CommonStats stats) {
+        this.skippedInactive.incrBy(stats.getRate());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java
new file mode 100644
index 0000000..ba7f9db
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon.supervisor;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Writer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.storm.Config;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AdvancedFSOps implements IAdvancedFSOps {
+    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
+    
+    /**
+     * Factory to create a new AdvancedFSOps
+     * @param conf the configuration of the process
+     * @return the appropriate instance of the class for this config and environment.
+     */
+    public static AdvancedFSOps make(Map<String, Object> conf) {
+        if (Utils.isOnWindows()) {
+            return new AdvancedWindowsFSOps(conf);
+        }
+        if (ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
+            return new AdvancedRunAsUserFSOps(conf);
+        }
+        return new AdvancedFSOps(conf);
+    }
+    
+    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
+        private final Map<String, Object> _conf;
+        
+        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
+            super(conf);
+            if (Utils.isOnWindows()) {
+                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
+            }
+            _conf = conf;
+        }
+        
+        @Override
+        public void setupBlobPermissions(File path, String user) throws IOException {
+            String logPrefix = "setup blob permissions for " + path;
+            ClientSupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
+        }
+        
+        @Override
+        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
+            String absolutePath = path.getAbsolutePath();
+            LOG.info("Deleting path {}", absolutePath);
+            if (user == null) {
+                user = Files.getOwner(path.toPath()).getName();
+            }
+            List<String> commands = new ArrayList<>();
+            commands.add("rmr");
+            commands.add(absolutePath);
+            ClientSupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
+
+            if (Utils.checkFileExists(absolutePath)) {
+                // It's possible that permissions were not set properly on the directory, and
+                // the user who is *supposed* to own the dir does not. In this case, try the
+                // delete as the supervisor user.
+                Utils.forceDelete(absolutePath);
+                if (Utils.checkFileExists(absolutePath)) {
+                    throw new RuntimeException(path + " was not deleted.");
+                }
+            }
+        }
+        
+        @Override
+        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
+            ClientSupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
+        }
+
+        @Override
+        public void setupWorkerArtifactsDir(Map<String, Object> topologyConf, File path) throws IOException {
+            ClientSupervisorUtils.setupWorkerArtifactsDir(_conf, topologyConf, path.getCanonicalPath());
+        }
+    }
+    
+    /**
+     * Operations that need to override the default ones when running on Windows
+     *
+     */
+    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
+
+        public AdvancedWindowsFSOps(Map<String, Object> conf) {
+            super(conf);
+            if (ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
+                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
+            }
+        }
+        
+        @Override
+        public void restrictDirectoryPermissions(File dir) throws IOException {
+            //NOOP, if windows gets support for run as user we will need to find a way to support this
+        }
+        
+        @Override
+        public void moveDirectoryPreferAtomic(File fromDir, File toDir) throws IOException {
+            // Files/move with non-empty directory doesn't work well on Windows
+            // This is not atomic but it does work
+            FileUtils.moveDirectory(fromDir, toDir);
+        }
+        
+        @Override
+        public boolean supportsAtomicDirectoryMove() {
+            // Files/move with non-empty directory doesn't work well on Windows
+            // FileUtils.moveDirectory is not atomic
+            return false;
+        }
+    }
+
+    protected final boolean _symlinksDisabled;
+    
+    protected AdvancedFSOps(Map<String, Object> conf) {
+        _symlinksDisabled = (boolean)conf.getOrDefault(Config.DISABLE_SYMLINKS, false);
+    }
+
+    /**
+     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
+     * On some systems that do not support this, it may become a noop
+     * @param dir the directory to change permissions on
+     * @throws IOException on any error
+     */
+    public void restrictDirectoryPermissions(File dir) throws IOException {
+        Set<PosixFilePermission> perms = new HashSet<>(
+                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
+                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
+                        PosixFilePermission.GROUP_EXECUTE));
+        Files.setPosixFilePermissions(dir.toPath(), perms);
+    }
+
+    /**
+     * Move fromDir to toDir, and try to make it an atomic move if possible
+     * @param fromDir what to move
+     * @param toDir where to move it from
+     * @throws IOException on any error
+     */
+    public void moveDirectoryPreferAtomic(File fromDir, File toDir) throws IOException {
+        FileUtils.forceMkdir(toDir);
+        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
+    }
+
+    /**
+     * @return true if an atomic directory move works, else false.
+     */
+    public boolean supportsAtomicDirectoryMove() {
+        return true;
+    }
+
+    /**
+     * Copy a directory
+     * @param fromDir from where
+     * @param toDir to where
+     * @throws IOException on any error
+     */
+    public void copyDirectory(File fromDir, File toDir) throws IOException {
+        FileUtils.copyDirectory(fromDir, toDir);
+    }
+
+    /**
+     * Setup permissions properly for an internal blob store path
+     * @param path the path to set the permissions on
+     * @param user the user to change the permissions for
+     * @throws IOException on any error
+     */
+    public void setupBlobPermissions(File path, String user) throws IOException {
+        //Normally this is a NOOP
+    }
+
+    /**
+     * Delete a file or a directory and all of the children. If it exists.
+     * @param path what to delete
+     * @param user who to delete it as if doing it as someone else is supported
+     * @param logPrefix if an external process needs to be launched to delete
+     * the object what prefix to include in the logs
+     * @throws IOException on any error.
+     */
+    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
+        //by default no need to do this as a different user
+        deleteIfExists(path);
+    }
+
+    /**
+     * Delete a file or a directory and all of the children. If it exists.
+     * @param path what to delete
+     * @throws IOException on any error.
+     */
+    public void deleteIfExists(File path) throws IOException {
+        LOG.info("Deleting path {}", path);
+        Path p = path.toPath();
+        if (Files.exists(p)) {
+            try {
+                FileUtils.forceDelete(path);
+            } catch (FileNotFoundException ignored) {}
+        }
+    }
+
+    /**
+     * Setup the permissions for the storm code dir
+     * @param topologyConf the config of the Topology
+     * @param path the directory to set the permissions on
+     * @throws IOException on any error
+     */
+    public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
+        //By default this is a NOOP
+    }
+
+    /**
+     * Setup the permissions for the worker artifacts dirs
+     * @param topologyConf the config of the Topology
+     * @param path the directory to set the permissions on
+     * @throws IOException on any error
+     */
+    public void setupWorkerArtifactsDir(Map<String, Object> topologyConf, File path) throws IOException {
+        //By default this is a NOOP
+    }
+
+    /**
+     * Sanity check if everything the topology needs is there for it to run.
+     * @param conf the config of the supervisor
+     * @param topologyId the ID of the topology
+     * @return true if everything is there, else false
+     * @throws IOException on any error
+     */
+    public boolean doRequiredTopoFilesExist(Map<String, Object> conf, String topologyId) throws IOException {
+        return ClientSupervisorUtils.doRequiredTopoFilesExist(conf, topologyId);
+    }
+
+    /**
+     * Makes a directory, including any necessary but nonexistent parent
+     * directories.
+     *
+     * @param path the directory to create
+     * @throws IOException on any error
+     */
+    public void forceMkdir(File path) throws IOException {
+        FileUtils.forceMkdir(path);
+    }
+
+    /**
+     * Check if a file exists or not
+     * @param path the path to check
+     * @return true if it exists else false
+     * @throws IOException on any error.
+     */
+    public boolean fileExists(File path) throws IOException {
+        return path.exists();
+    }
+
+    /**
+     * Get a writer for the given location
+     * @param file the file to write to
+     * @return the Writer to use.
+     * @throws IOException on any error
+     */
+    public Writer getWriter(File file) throws IOException {
+        return new FileWriter(file);
+    }
+
+    /**
+     * Get an output stream to write to a given file
+     * @param file the file to write to
+     * @return an OutputStream for that file
+     * @throws IOException on any error
+     */
+    public OutputStream getOutputStream(File file) throws IOException {
+        return new FileOutputStream(file);
+    }
+
+    /**
+     * Dump a string to a file
+     * @param location where to write to
+     * @param data the data to write
+     * @throws IOException on any error
+     */
+    public void dump(File location, String data) throws IOException {
+        File parent = location.getParentFile();
+        if (!parent.exists()) {
+            forceMkdir(parent);
+        }
+        try (Writer w = getWriter(location)) {
+            w.write(data);
+        }
+    }
+
+    /**
+     * Read the contents of a file into a String
+     * @param location the file to read
+     * @return the contents of the file
+     * @throws IOException on any error
+     */
+    @Override
+    public String slurpString(File location) throws IOException {
+        return FileUtils.readFileToString(location, "UTF-8");
+    }
+
+    /**
+     * Read the contents of a file into a byte array.
+     * @param location the file to read
+     * @return the contents of the file
+     * @throws IOException on any error
+     */
+    @Override
+    public byte[] slurp(File location) throws IOException {
+        return FileUtils.readFileToByteArray(location);
+    }
+
+    /**
+     * Create a symbolic link pointing at target
+     * @param link the link to create
+     * @param target where it should point to
+     * @throws IOException on any error.
+     */
+    @Override
+    public void createSymlink(File link, File target) throws IOException {
+        if (_symlinksDisabled) {
+            throw new IOException("Symlinks have been disabled, this should not be called");
+        }
+        Path plink = link.toPath().toAbsolutePath();
+        Path ptarget = target.toPath().toAbsolutePath();
+        LOG.debug("Creating symlink [{}] to [{}]", plink, ptarget);
+        if (Files.exists(plink)) {
+            if (Files.isSameFile(plink, ptarget)) {
+                //It already points where we want it to
+                return;
+            }
+            FileUtils.forceDelete(link);
+        }
+        Files.createSymbolicLink(plink, ptarget);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java
new file mode 100644
index 0000000..8cd7261
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.daemon.supervisor;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.storm.Config;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.ObjectReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+public class ClientSupervisorUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(ClientSupervisorUtils.class);
+
+    static boolean doRequiredTopoFilesExist(Map<String, Object> conf, String stormId) throws IOException {
+        String stormroot = ConfigUtils.supervisorStormDistRoot(conf, stormId);
+        String stormjarpath = ConfigUtils.supervisorStormJarPath(stormroot);
+        String stormcodepath = ConfigUtils.supervisorStormCodePath(stormroot);
+        String stormconfpath = ConfigUtils.supervisorStormConfPath(stormroot);
+        if (!Utils.checkFileExists(stormroot))
+            return false;
+        if (!Utils.checkFileExists(stormcodepath))
+            return false;
+        if (!Utils.checkFileExists(stormconfpath))
+            return false;
+        if (ConfigUtils.isLocalMode(conf) || Utils.checkFileExists(stormjarpath))
+            return true;
+        return false;
+    }
+
+    public static int processLauncherAndWait(Map<String, Object> conf, String user, List<String> args, final Map<String, String> environment, final String logPreFix)
+            throws IOException {
+        int ret = 0;
+        Process process = processLauncher(conf, user, null, args, environment, logPreFix, null, null);
+        if (StringUtils.isNotBlank(logPreFix))
+            Utils.readAndLogStream(logPreFix, process.getInputStream());
+        try {
+            process.waitFor();
+        } catch (InterruptedException e) {
+            LOG.info("{} interrupted.", logPreFix);
+        }
+        ret = process.exitValue();
+        return ret;
+    }
+
+    static Process processLauncher(Map<String, Object> conf, String user, List<String> commandPrefix, List<String> args, Map<String, String> environment, final String logPreFix,
+                                          final ExitCodeCallback exitCodeCallback, File dir) throws IOException {
+        if (StringUtils.isBlank(user)) {
+            throw new IllegalArgumentException("User cannot be blank when calling processLauncher.");
+        }
+        String wlinitial = (String) (conf.get(Config.SUPERVISOR_WORKER_LAUNCHER));
+        String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
+        String wl;
+        if (StringUtils.isNotBlank(wlinitial)) {
+            wl = wlinitial;
+        } else {
+            wl = stormHome + "/bin/worker-launcher";
+        }
+        List<String> commands = new ArrayList<>();
+        if (commandPrefix != null){
+            commands.addAll(commandPrefix);
+        }
+        commands.add(wl);
+        commands.add(user);
+        commands.addAll(args);
+        LOG.info("Running as user: {} command: {}", user, commands);
+        return launchProcess(commands, environment, logPreFix, exitCodeCallback, dir);
+    }
+
+    /**
+     * Launch a new process as per {@link ProcessBuilder} with a given
+     * callback.
+     * @param command the command to be executed in the new process
+     * @param environment the environment to be applied to the process. Can be
+     *                    null.
+     * @param logPrefix a prefix for log entries from the output of the process.
+     *                  Can be null.
+     * @param exitCodeCallback code to be called passing the exit code value
+     *                         when the process completes
+     * @param dir the working directory of the new process
+     * @return the new process
+     * @throws IOException
+     * @see ProcessBuilder
+     */
+    public static Process launchProcess(List<String> command,
+                                        Map<String,String> environment,
+                                        final String logPrefix,
+                                        final ExitCodeCallback exitCodeCallback,
+                                        File dir)
+            throws IOException {
+        ProcessBuilder builder = new ProcessBuilder(command);
+        Map<String,String> procEnv = builder.environment();
+        if (dir != null) {
+            builder.directory(dir);
+        }
+        builder.redirectErrorStream(true);
+        if (environment != null) {
+            procEnv.putAll(environment);
+        }
+        final Process process = builder.start();
+        if (logPrefix != null || exitCodeCallback != null) {
+            Utils.asyncLoop(new Callable<Object>() {
+                public Object call() {
+                    if (logPrefix != null ) {
+                        Utils.readAndLogStream(logPrefix,
+                                process.getInputStream());
+                    }
+                    if (exitCodeCallback != null) {
+                        try {
+                            process.waitFor();
+                            exitCodeCallback.call(process.exitValue());
+                        } catch (InterruptedException ie) {
+                            LOG.info("{} interrupted", logPrefix);
+                            exitCodeCallback.call(-1);
+                        }
+                    }
+                    return null; // Run only once.
+                }
+            });
+        }
+        return process;
+    }
+
+    public static void setupStormCodeDir(Map<String, Object> conf, Map<String, Object> stormConf, String dir) throws IOException {
+        if (ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
+            String logPrefix = "Storm Code Dir Setup for " + dir;
+            List<String> commands = new ArrayList<>();
+            commands.add("code-dir");
+            commands.add(dir);
+            processLauncherAndWait(conf, (String) (stormConf.get(Config.TOPOLOGY_SUBMITTER_USER)), commands, null, logPrefix);
+        }
+    }
+
+    public static void setupWorkerArtifactsDir(Map<String, Object> conf, Map<String, Object> stormConf, String dir) throws IOException {
+        if (ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
+            String logPrefix = "Worker Artifacts Setup for " + dir;
+            List<String> commands = new ArrayList<>();
+            commands.add("artifacts-dir");
+            commands.add(dir);
+            processLauncherAndWait(conf, (String) (stormConf.get(Config.TOPOLOGY_SUBMITTER_USER)), commands, null, logPrefix);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java
new file mode 100644
index 0000000..082f205
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.daemon.supervisor;
+
+/**
+ * A callback that can accept an integer.
+ */
+public interface ExitCodeCallback {
+    
+    /**
+     * The process finished 
+     * @param exitCode the exit code of the finished process.
+     */
+    public void call(int exitCode);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java
new file mode 100644
index 0000000..e5f5db0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.daemon.supervisor;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Writer;
+import java.util.Map;
+
+public interface IAdvancedFSOps {
+
+    /**
+     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
+     * On some systems that do not support this, it may become a noop
+     * @param dir the directory to change permissions on
+     * @throws IOException on any error
+     */
+    void restrictDirectoryPermissions(File dir) throws IOException;
+
+    /**
+     * Move fromDir to toDir, and try to make it an atomic move if possible
+     * @param fromDir what to move
+     * @param toDir where to move it from
+     * @throws IOException on any error
+     */
+    void moveDirectoryPreferAtomic(File fromDir, File toDir) throws IOException;
+
+    /**
+     * @return true if an atomic directory move works, else false.
+     */
+    boolean supportsAtomicDirectoryMove();
+
+    /**
+     * Copy a directory
+     * @param fromDir from where
+     * @param toDir to where
+     * @throws IOException on any error
+     */
+    void copyDirectory(File fromDir, File toDir) throws IOException;
+
+    /**
+     * Setup permissions properly for an internal blob store path
+     * @param path the path to set the permissions on
+     * @param user the user to change the permissions for
+     * @throws IOException on any error
+     */
+    void setupBlobPermissions(File path, String user) throws IOException;
+
+    /**
+     * Delete a file or a directory and all of the children. If it exists.
+     * @param path what to delete
+     * @param user who to delete it as if doing it as someone else is supported
+     * @param logPrefix if an external process needs to be launched to delete
+     * the object what prefix to include in the logs
+     * @throws IOException on any error.
+     */
+    void deleteIfExists(File path, String user, String logPrefix) throws IOException;
+
+    /**
+     * Delete a file or a directory and all of the children. If it exists.
+     * @param path what to delete
+     * @throws IOException on any error.
+     */
+    void deleteIfExists(File path) throws IOException;
+
+    /**
+     * Setup the permissions for the storm code dir
+     * @param topologyConf the config of the Topology
+     * @param path the directory to set the permissions on
+     * @throws IOException on any error
+     */
+    void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException;
+
+    /**
+     * Setup the permissions for the worker artifacts dirs
+     * @param topologyConf the config of the Topology
+     * @param path the directory to set the permissions on
+     * @throws IOException on any error
+     */
+    void setupWorkerArtifactsDir(Map<String, Object> topologyConf, File path) throws IOException;
+
+    /**
+     * Sanity check if everything the topology needs is there for it to run.
+     * @param conf the config of the supervisor
+     * @param topologyId the ID of the topology
+     * @return true if everything is there, else false
+     * @throws IOException on any error
+     */
+    boolean doRequiredTopoFilesExist(Map<String, Object> conf, String topologyId) throws IOException;
+
+    /**
+     * Makes a directory, including any necessary but nonexistent parent
+     * directories.
+     *
+     * @param path the directory to create
+     * @throws IOException on any error
+     */
+    void forceMkdir(File path) throws IOException;
+
+    /**
+     * Check if a file exists or not
+     * @param path the path to check
+     * @return true if it exists else false
+     * @throws IOException on any error.
+     */
+    boolean fileExists(File path) throws IOException;
+
+    /**
+     * Get a writer for the given location
+     * @param file the file to write to
+     * @return the Writer to use.
+     * @throws IOException on any error
+     */
+    Writer getWriter(File file) throws IOException;
+
+    /**
+     * Get an output stream to write to a given file
+     * @param file the file to write to
+     * @return an OutputStream for that file
+     * @throws IOException on any error
+     */
+    OutputStream getOutputStream(File file) throws IOException;
+
+    /**
+     * Dump a string to a file
+     * @param location where to write to
+     * @param data the data to write
+     * @throws IOException on any error
+     */
+    void dump(File location, String data) throws IOException;
+
+    /**
+     * Read the contents of a file into a String
+     * @param location the file to read
+     * @return the contents of the file
+     * @throws IOException on any error
+     */
+    String slurpString(File location) throws IOException;
+
+    /**
+     * Read the contents of a file into a byte array.
+     * @param localtion the file to read
+     * @return the contents of the file
+     * @throws IOException on any error
+     */
+    byte[] slurp(File location) throws IOException;
+
+    /**
+     * Create a symbolic link pointing at target
+     * @param link the link to create
+     * @param target where it should point to
+     * @throws IOException on any error.
+     */
+    void createSymlink(File link, File target) throws IOException;
+}


[32/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ComponentPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ComponentPageInfo.java b/storm-client/src/jvm/org/apache/storm/generated/ComponentPageInfo.java
new file mode 100644
index 0000000..7fcebcc
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ComponentPageInfo.java
@@ -0,0 +1,2352 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageInfo, ComponentPageInfo._Fields>, java.io.Serializable, Cloneable, Comparable<ComponentPageInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentPageInfo");
+
+  private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField COMPONENT_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("component_type", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_stats", org.apache.thrift.protocol.TType.MAP, (short)7);
+  private static final org.apache.thrift.protocol.TField GSID_TO_INPUT_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("gsid_to_input_stats", org.apache.thrift.protocol.TType.MAP, (short)8);
+  private static final org.apache.thrift.protocol.TField SID_TO_OUTPUT_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("sid_to_output_stats", org.apache.thrift.protocol.TType.MAP, (short)9);
+  private static final org.apache.thrift.protocol.TField EXEC_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("exec_stats", org.apache.thrift.protocol.TType.LIST, (short)10);
+  private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.LIST, (short)11);
+  private static final org.apache.thrift.protocol.TField EVENTLOG_HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("eventlog_host", org.apache.thrift.protocol.TType.STRING, (short)12);
+  private static final org.apache.thrift.protocol.TField EVENTLOG_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("eventlog_port", org.apache.thrift.protocol.TType.I32, (short)13);
+  private static final org.apache.thrift.protocol.TField DEBUG_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("debug_options", org.apache.thrift.protocol.TType.STRUCT, (short)14);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_status", org.apache.thrift.protocol.TType.STRING, (short)15);
+  private static final org.apache.thrift.protocol.TField RESOURCES_MAP_FIELD_DESC = new org.apache.thrift.protocol.TField("resources_map", org.apache.thrift.protocol.TType.MAP, (short)16);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ComponentPageInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ComponentPageInfoTupleSchemeFactory());
+  }
+
+  private String component_id; // required
+  private ComponentType component_type; // required
+  private String topology_id; // optional
+  private String topology_name; // optional
+  private int num_executors; // optional
+  private int num_tasks; // optional
+  private Map<String,ComponentAggregateStats> window_to_stats; // optional
+  private Map<GlobalStreamId,ComponentAggregateStats> gsid_to_input_stats; // optional
+  private Map<String,ComponentAggregateStats> sid_to_output_stats; // optional
+  private List<ExecutorAggregateStats> exec_stats; // optional
+  private List<ErrorInfo> errors; // optional
+  private String eventlog_host; // optional
+  private int eventlog_port; // optional
+  private DebugOptions debug_options; // optional
+  private String topology_status; // optional
+  private Map<String,Double> resources_map; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    COMPONENT_ID((short)1, "component_id"),
+    /**
+     * 
+     * @see ComponentType
+     */
+    COMPONENT_TYPE((short)2, "component_type"),
+    TOPOLOGY_ID((short)3, "topology_id"),
+    TOPOLOGY_NAME((short)4, "topology_name"),
+    NUM_EXECUTORS((short)5, "num_executors"),
+    NUM_TASKS((short)6, "num_tasks"),
+    WINDOW_TO_STATS((short)7, "window_to_stats"),
+    GSID_TO_INPUT_STATS((short)8, "gsid_to_input_stats"),
+    SID_TO_OUTPUT_STATS((short)9, "sid_to_output_stats"),
+    EXEC_STATS((short)10, "exec_stats"),
+    ERRORS((short)11, "errors"),
+    EVENTLOG_HOST((short)12, "eventlog_host"),
+    EVENTLOG_PORT((short)13, "eventlog_port"),
+    DEBUG_OPTIONS((short)14, "debug_options"),
+    TOPOLOGY_STATUS((short)15, "topology_status"),
+    RESOURCES_MAP((short)16, "resources_map");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // COMPONENT_ID
+          return COMPONENT_ID;
+        case 2: // COMPONENT_TYPE
+          return COMPONENT_TYPE;
+        case 3: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 4: // TOPOLOGY_NAME
+          return TOPOLOGY_NAME;
+        case 5: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 6: // NUM_TASKS
+          return NUM_TASKS;
+        case 7: // WINDOW_TO_STATS
+          return WINDOW_TO_STATS;
+        case 8: // GSID_TO_INPUT_STATS
+          return GSID_TO_INPUT_STATS;
+        case 9: // SID_TO_OUTPUT_STATS
+          return SID_TO_OUTPUT_STATS;
+        case 10: // EXEC_STATS
+          return EXEC_STATS;
+        case 11: // ERRORS
+          return ERRORS;
+        case 12: // EVENTLOG_HOST
+          return EVENTLOG_HOST;
+        case 13: // EVENTLOG_PORT
+          return EVENTLOG_PORT;
+        case 14: // DEBUG_OPTIONS
+          return DEBUG_OPTIONS;
+        case 15: // TOPOLOGY_STATUS
+          return TOPOLOGY_STATUS;
+        case 16: // RESOURCES_MAP
+          return RESOURCES_MAP;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_EXECUTORS_ISSET_ID = 0;
+  private static final int __NUM_TASKS_ISSET_ID = 1;
+  private static final int __EVENTLOG_PORT_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.TOPOLOGY_ID,_Fields.TOPOLOGY_NAME,_Fields.NUM_EXECUTORS,_Fields.NUM_TASKS,_Fields.WINDOW_TO_STATS,_Fields.GSID_TO_INPUT_STATS,_Fields.SID_TO_OUTPUT_STATS,_Fields.EXEC_STATS,_Fields.ERRORS,_Fields.EVENTLOG_HOST,_Fields.EVENTLOG_PORT,_Fields.DEBUG_OPTIONS,_Fields.TOPOLOGY_STATUS,_Fields.RESOURCES_MAP};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COMPONENT_TYPE, new org.apache.thrift.meta_data.FieldMetaData("component_type", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ComponentType.class)));
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topology_name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.WINDOW_TO_STATS, new org.apache.thrift.meta_data.FieldMetaData("window_to_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.GSID_TO_INPUT_STATS, new org.apache.thrift.meta_data.FieldMetaData("gsid_to_input_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.SID_TO_OUTPUT_STATS, new org.apache.thrift.meta_data.FieldMetaData("sid_to_output_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.EXEC_STATS, new org.apache.thrift.meta_data.FieldMetaData("exec_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorAggregateStats.class))));
+    tmpMap.put(_Fields.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class))));
+    tmpMap.put(_Fields.EVENTLOG_HOST, new org.apache.thrift.meta_data.FieldMetaData("eventlog_host", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EVENTLOG_PORT, new org.apache.thrift.meta_data.FieldMetaData("eventlog_port", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.DEBUG_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("debug_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class)));
+    tmpMap.put(_Fields.TOPOLOGY_STATUS, new org.apache.thrift.meta_data.FieldMetaData("topology_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.RESOURCES_MAP, new org.apache.thrift.meta_data.FieldMetaData("resources_map", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentPageInfo.class, metaDataMap);
+  }
+
+  public ComponentPageInfo() {
+  }
+
+  public ComponentPageInfo(
+    String component_id,
+    ComponentType component_type)
+  {
+    this();
+    this.component_id = component_id;
+    this.component_type = component_type;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ComponentPageInfo(ComponentPageInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_component_id()) {
+      this.component_id = other.component_id;
+    }
+    if (other.is_set_component_type()) {
+      this.component_type = other.component_type;
+    }
+    if (other.is_set_topology_id()) {
+      this.topology_id = other.topology_id;
+    }
+    if (other.is_set_topology_name()) {
+      this.topology_name = other.topology_name;
+    }
+    this.num_executors = other.num_executors;
+    this.num_tasks = other.num_tasks;
+    if (other.is_set_window_to_stats()) {
+      Map<String,ComponentAggregateStats> __this__window_to_stats = new HashMap<String,ComponentAggregateStats>(other.window_to_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.window_to_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__window_to_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__window_to_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__window_to_stats.put(__this__window_to_stats_copy_key, __this__window_to_stats_copy_value);
+      }
+      this.window_to_stats = __this__window_to_stats;
+    }
+    if (other.is_set_gsid_to_input_stats()) {
+      Map<GlobalStreamId,ComponentAggregateStats> __this__gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(other.gsid_to_input_stats.size());
+      for (Map.Entry<GlobalStreamId, ComponentAggregateStats> other_element : other.gsid_to_input_stats.entrySet()) {
+
+        GlobalStreamId other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        GlobalStreamId __this__gsid_to_input_stats_copy_key = new GlobalStreamId(other_element_key);
+
+        ComponentAggregateStats __this__gsid_to_input_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__gsid_to_input_stats.put(__this__gsid_to_input_stats_copy_key, __this__gsid_to_input_stats_copy_value);
+      }
+      this.gsid_to_input_stats = __this__gsid_to_input_stats;
+    }
+    if (other.is_set_sid_to_output_stats()) {
+      Map<String,ComponentAggregateStats> __this__sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(other.sid_to_output_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.sid_to_output_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__sid_to_output_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__sid_to_output_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__sid_to_output_stats.put(__this__sid_to_output_stats_copy_key, __this__sid_to_output_stats_copy_value);
+      }
+      this.sid_to_output_stats = __this__sid_to_output_stats;
+    }
+    if (other.is_set_exec_stats()) {
+      List<ExecutorAggregateStats> __this__exec_stats = new ArrayList<ExecutorAggregateStats>(other.exec_stats.size());
+      for (ExecutorAggregateStats other_element : other.exec_stats) {
+        __this__exec_stats.add(new ExecutorAggregateStats(other_element));
+      }
+      this.exec_stats = __this__exec_stats;
+    }
+    if (other.is_set_errors()) {
+      List<ErrorInfo> __this__errors = new ArrayList<ErrorInfo>(other.errors.size());
+      for (ErrorInfo other_element : other.errors) {
+        __this__errors.add(new ErrorInfo(other_element));
+      }
+      this.errors = __this__errors;
+    }
+    if (other.is_set_eventlog_host()) {
+      this.eventlog_host = other.eventlog_host;
+    }
+    this.eventlog_port = other.eventlog_port;
+    if (other.is_set_debug_options()) {
+      this.debug_options = new DebugOptions(other.debug_options);
+    }
+    if (other.is_set_topology_status()) {
+      this.topology_status = other.topology_status;
+    }
+    if (other.is_set_resources_map()) {
+      Map<String,Double> __this__resources_map = new HashMap<String,Double>(other.resources_map);
+      this.resources_map = __this__resources_map;
+    }
+  }
+
+  public ComponentPageInfo deepCopy() {
+    return new ComponentPageInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.component_id = null;
+    this.component_type = null;
+    this.topology_id = null;
+    this.topology_name = null;
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    set_num_tasks_isSet(false);
+    this.num_tasks = 0;
+    this.window_to_stats = null;
+    this.gsid_to_input_stats = null;
+    this.sid_to_output_stats = null;
+    this.exec_stats = null;
+    this.errors = null;
+    this.eventlog_host = null;
+    set_eventlog_port_isSet(false);
+    this.eventlog_port = 0;
+    this.debug_options = null;
+    this.topology_status = null;
+    this.resources_map = null;
+  }
+
+  public String get_component_id() {
+    return this.component_id;
+  }
+
+  public void set_component_id(String component_id) {
+    this.component_id = component_id;
+  }
+
+  public void unset_component_id() {
+    this.component_id = null;
+  }
+
+  /** Returns true if field component_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_id() {
+    return this.component_id != null;
+  }
+
+  public void set_component_id_isSet(boolean value) {
+    if (!value) {
+      this.component_id = null;
+    }
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public ComponentType get_component_type() {
+    return this.component_type;
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public void set_component_type(ComponentType component_type) {
+    this.component_type = component_type;
+  }
+
+  public void unset_component_type() {
+    this.component_type = null;
+  }
+
+  /** Returns true if field component_type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_type() {
+    return this.component_type != null;
+  }
+
+  public void set_component_type_isSet(boolean value) {
+    if (!value) {
+      this.component_type = null;
+    }
+  }
+
+  public String get_topology_id() {
+    return this.topology_id;
+  }
+
+  public void set_topology_id(String topology_id) {
+    this.topology_id = topology_id;
+  }
+
+  public void unset_topology_id() {
+    this.topology_id = null;
+  }
+
+  /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_id() {
+    return this.topology_id != null;
+  }
+
+  public void set_topology_id_isSet(boolean value) {
+    if (!value) {
+      this.topology_id = null;
+    }
+  }
+
+  public String get_topology_name() {
+    return this.topology_name;
+  }
+
+  public void set_topology_name(String topology_name) {
+    this.topology_name = topology_name;
+  }
+
+  public void unset_topology_name() {
+    this.topology_name = null;
+  }
+
+  /** Returns true if field topology_name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_name() {
+    return this.topology_name != null;
+  }
+
+  public void set_topology_name_isSet(boolean value) {
+    if (!value) {
+      this.topology_name = null;
+    }
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value);
+  }
+
+  public int get_window_to_stats_size() {
+    return (this.window_to_stats == null) ? 0 : this.window_to_stats.size();
+  }
+
+  public void put_to_window_to_stats(String key, ComponentAggregateStats val) {
+    if (this.window_to_stats == null) {
+      this.window_to_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.window_to_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_window_to_stats() {
+    return this.window_to_stats;
+  }
+
+  public void set_window_to_stats(Map<String,ComponentAggregateStats> window_to_stats) {
+    this.window_to_stats = window_to_stats;
+  }
+
+  public void unset_window_to_stats() {
+    this.window_to_stats = null;
+  }
+
+  /** Returns true if field window_to_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_stats() {
+    return this.window_to_stats != null;
+  }
+
+  public void set_window_to_stats_isSet(boolean value) {
+    if (!value) {
+      this.window_to_stats = null;
+    }
+  }
+
+  public int get_gsid_to_input_stats_size() {
+    return (this.gsid_to_input_stats == null) ? 0 : this.gsid_to_input_stats.size();
+  }
+
+  public void put_to_gsid_to_input_stats(GlobalStreamId key, ComponentAggregateStats val) {
+    if (this.gsid_to_input_stats == null) {
+      this.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>();
+    }
+    this.gsid_to_input_stats.put(key, val);
+  }
+
+  public Map<GlobalStreamId,ComponentAggregateStats> get_gsid_to_input_stats() {
+    return this.gsid_to_input_stats;
+  }
+
+  public void set_gsid_to_input_stats(Map<GlobalStreamId,ComponentAggregateStats> gsid_to_input_stats) {
+    this.gsid_to_input_stats = gsid_to_input_stats;
+  }
+
+  public void unset_gsid_to_input_stats() {
+    this.gsid_to_input_stats = null;
+  }
+
+  /** Returns true if field gsid_to_input_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_gsid_to_input_stats() {
+    return this.gsid_to_input_stats != null;
+  }
+
+  public void set_gsid_to_input_stats_isSet(boolean value) {
+    if (!value) {
+      this.gsid_to_input_stats = null;
+    }
+  }
+
+  public int get_sid_to_output_stats_size() {
+    return (this.sid_to_output_stats == null) ? 0 : this.sid_to_output_stats.size();
+  }
+
+  public void put_to_sid_to_output_stats(String key, ComponentAggregateStats val) {
+    if (this.sid_to_output_stats == null) {
+      this.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.sid_to_output_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_sid_to_output_stats() {
+    return this.sid_to_output_stats;
+  }
+
+  public void set_sid_to_output_stats(Map<String,ComponentAggregateStats> sid_to_output_stats) {
+    this.sid_to_output_stats = sid_to_output_stats;
+  }
+
+  public void unset_sid_to_output_stats() {
+    this.sid_to_output_stats = null;
+  }
+
+  /** Returns true if field sid_to_output_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sid_to_output_stats() {
+    return this.sid_to_output_stats != null;
+  }
+
+  public void set_sid_to_output_stats_isSet(boolean value) {
+    if (!value) {
+      this.sid_to_output_stats = null;
+    }
+  }
+
+  public int get_exec_stats_size() {
+    return (this.exec_stats == null) ? 0 : this.exec_stats.size();
+  }
+
+  public java.util.Iterator<ExecutorAggregateStats> get_exec_stats_iterator() {
+    return (this.exec_stats == null) ? null : this.exec_stats.iterator();
+  }
+
+  public void add_to_exec_stats(ExecutorAggregateStats elem) {
+    if (this.exec_stats == null) {
+      this.exec_stats = new ArrayList<ExecutorAggregateStats>();
+    }
+    this.exec_stats.add(elem);
+  }
+
+  public List<ExecutorAggregateStats> get_exec_stats() {
+    return this.exec_stats;
+  }
+
+  public void set_exec_stats(List<ExecutorAggregateStats> exec_stats) {
+    this.exec_stats = exec_stats;
+  }
+
+  public void unset_exec_stats() {
+    this.exec_stats = null;
+  }
+
+  /** Returns true if field exec_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_exec_stats() {
+    return this.exec_stats != null;
+  }
+
+  public void set_exec_stats_isSet(boolean value) {
+    if (!value) {
+      this.exec_stats = null;
+    }
+  }
+
+  public int get_errors_size() {
+    return (this.errors == null) ? 0 : this.errors.size();
+  }
+
+  public java.util.Iterator<ErrorInfo> get_errors_iterator() {
+    return (this.errors == null) ? null : this.errors.iterator();
+  }
+
+  public void add_to_errors(ErrorInfo elem) {
+    if (this.errors == null) {
+      this.errors = new ArrayList<ErrorInfo>();
+    }
+    this.errors.add(elem);
+  }
+
+  public List<ErrorInfo> get_errors() {
+    return this.errors;
+  }
+
+  public void set_errors(List<ErrorInfo> errors) {
+    this.errors = errors;
+  }
+
+  public void unset_errors() {
+    this.errors = null;
+  }
+
+  /** Returns true if field errors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_errors() {
+    return this.errors != null;
+  }
+
+  public void set_errors_isSet(boolean value) {
+    if (!value) {
+      this.errors = null;
+    }
+  }
+
+  public String get_eventlog_host() {
+    return this.eventlog_host;
+  }
+
+  public void set_eventlog_host(String eventlog_host) {
+    this.eventlog_host = eventlog_host;
+  }
+
+  public void unset_eventlog_host() {
+    this.eventlog_host = null;
+  }
+
+  /** Returns true if field eventlog_host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_eventlog_host() {
+    return this.eventlog_host != null;
+  }
+
+  public void set_eventlog_host_isSet(boolean value) {
+    if (!value) {
+      this.eventlog_host = null;
+    }
+  }
+
+  public int get_eventlog_port() {
+    return this.eventlog_port;
+  }
+
+  public void set_eventlog_port(int eventlog_port) {
+    this.eventlog_port = eventlog_port;
+    set_eventlog_port_isSet(true);
+  }
+
+  public void unset_eventlog_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID);
+  }
+
+  /** Returns true if field eventlog_port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_eventlog_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID);
+  }
+
+  public void set_eventlog_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID, value);
+  }
+
+  public DebugOptions get_debug_options() {
+    return this.debug_options;
+  }
+
+  public void set_debug_options(DebugOptions debug_options) {
+    this.debug_options = debug_options;
+  }
+
+  public void unset_debug_options() {
+    this.debug_options = null;
+  }
+
+  /** Returns true if field debug_options is set (has been assigned a value) and false otherwise */
+  public boolean is_set_debug_options() {
+    return this.debug_options != null;
+  }
+
+  public void set_debug_options_isSet(boolean value) {
+    if (!value) {
+      this.debug_options = null;
+    }
+  }
+
+  public String get_topology_status() {
+    return this.topology_status;
+  }
+
+  public void set_topology_status(String topology_status) {
+    this.topology_status = topology_status;
+  }
+
+  public void unset_topology_status() {
+    this.topology_status = null;
+  }
+
+  /** Returns true if field topology_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_status() {
+    return this.topology_status != null;
+  }
+
+  public void set_topology_status_isSet(boolean value) {
+    if (!value) {
+      this.topology_status = null;
+    }
+  }
+
+  public int get_resources_map_size() {
+    return (this.resources_map == null) ? 0 : this.resources_map.size();
+  }
+
+  public void put_to_resources_map(String key, double val) {
+    if (this.resources_map == null) {
+      this.resources_map = new HashMap<String,Double>();
+    }
+    this.resources_map.put(key, val);
+  }
+
+  public Map<String,Double> get_resources_map() {
+    return this.resources_map;
+  }
+
+  public void set_resources_map(Map<String,Double> resources_map) {
+    this.resources_map = resources_map;
+  }
+
+  public void unset_resources_map() {
+    this.resources_map = null;
+  }
+
+  /** Returns true if field resources_map is set (has been assigned a value) and false otherwise */
+  public boolean is_set_resources_map() {
+    return this.resources_map != null;
+  }
+
+  public void set_resources_map_isSet(boolean value) {
+    if (!value) {
+      this.resources_map = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case COMPONENT_ID:
+      if (value == null) {
+        unset_component_id();
+      } else {
+        set_component_id((String)value);
+      }
+      break;
+
+    case COMPONENT_TYPE:
+      if (value == null) {
+        unset_component_type();
+      } else {
+        set_component_type((ComponentType)value);
+      }
+      break;
+
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topology_id();
+      } else {
+        set_topology_id((String)value);
+      }
+      break;
+
+    case TOPOLOGY_NAME:
+      if (value == null) {
+        unset_topology_name();
+      } else {
+        set_topology_name((String)value);
+      }
+      break;
+
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case WINDOW_TO_STATS:
+      if (value == null) {
+        unset_window_to_stats();
+      } else {
+        set_window_to_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case GSID_TO_INPUT_STATS:
+      if (value == null) {
+        unset_gsid_to_input_stats();
+      } else {
+        set_gsid_to_input_stats((Map<GlobalStreamId,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case SID_TO_OUTPUT_STATS:
+      if (value == null) {
+        unset_sid_to_output_stats();
+      } else {
+        set_sid_to_output_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case EXEC_STATS:
+      if (value == null) {
+        unset_exec_stats();
+      } else {
+        set_exec_stats((List<ExecutorAggregateStats>)value);
+      }
+      break;
+
+    case ERRORS:
+      if (value == null) {
+        unset_errors();
+      } else {
+        set_errors((List<ErrorInfo>)value);
+      }
+      break;
+
+    case EVENTLOG_HOST:
+      if (value == null) {
+        unset_eventlog_host();
+      } else {
+        set_eventlog_host((String)value);
+      }
+      break;
+
+    case EVENTLOG_PORT:
+      if (value == null) {
+        unset_eventlog_port();
+      } else {
+        set_eventlog_port((Integer)value);
+      }
+      break;
+
+    case DEBUG_OPTIONS:
+      if (value == null) {
+        unset_debug_options();
+      } else {
+        set_debug_options((DebugOptions)value);
+      }
+      break;
+
+    case TOPOLOGY_STATUS:
+      if (value == null) {
+        unset_topology_status();
+      } else {
+        set_topology_status((String)value);
+      }
+      break;
+
+    case RESOURCES_MAP:
+      if (value == null) {
+        unset_resources_map();
+      } else {
+        set_resources_map((Map<String,Double>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case COMPONENT_ID:
+      return get_component_id();
+
+    case COMPONENT_TYPE:
+      return get_component_type();
+
+    case TOPOLOGY_ID:
+      return get_topology_id();
+
+    case TOPOLOGY_NAME:
+      return get_topology_name();
+
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case NUM_TASKS:
+      return get_num_tasks();
+
+    case WINDOW_TO_STATS:
+      return get_window_to_stats();
+
+    case GSID_TO_INPUT_STATS:
+      return get_gsid_to_input_stats();
+
+    case SID_TO_OUTPUT_STATS:
+      return get_sid_to_output_stats();
+
+    case EXEC_STATS:
+      return get_exec_stats();
+
+    case ERRORS:
+      return get_errors();
+
+    case EVENTLOG_HOST:
+      return get_eventlog_host();
+
+    case EVENTLOG_PORT:
+      return get_eventlog_port();
+
+    case DEBUG_OPTIONS:
+      return get_debug_options();
+
+    case TOPOLOGY_STATUS:
+      return get_topology_status();
+
+    case RESOURCES_MAP:
+      return get_resources_map();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case COMPONENT_ID:
+      return is_set_component_id();
+    case COMPONENT_TYPE:
+      return is_set_component_type();
+    case TOPOLOGY_ID:
+      return is_set_topology_id();
+    case TOPOLOGY_NAME:
+      return is_set_topology_name();
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case WINDOW_TO_STATS:
+      return is_set_window_to_stats();
+    case GSID_TO_INPUT_STATS:
+      return is_set_gsid_to_input_stats();
+    case SID_TO_OUTPUT_STATS:
+      return is_set_sid_to_output_stats();
+    case EXEC_STATS:
+      return is_set_exec_stats();
+    case ERRORS:
+      return is_set_errors();
+    case EVENTLOG_HOST:
+      return is_set_eventlog_host();
+    case EVENTLOG_PORT:
+      return is_set_eventlog_port();
+    case DEBUG_OPTIONS:
+      return is_set_debug_options();
+    case TOPOLOGY_STATUS:
+      return is_set_topology_status();
+    case RESOURCES_MAP:
+      return is_set_resources_map();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ComponentPageInfo)
+      return this.equals((ComponentPageInfo)that);
+    return false;
+  }
+
+  public boolean equals(ComponentPageInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_component_id = true && this.is_set_component_id();
+    boolean that_present_component_id = true && that.is_set_component_id();
+    if (this_present_component_id || that_present_component_id) {
+      if (!(this_present_component_id && that_present_component_id))
+        return false;
+      if (!this.component_id.equals(that.component_id))
+        return false;
+    }
+
+    boolean this_present_component_type = true && this.is_set_component_type();
+    boolean that_present_component_type = true && that.is_set_component_type();
+    if (this_present_component_type || that_present_component_type) {
+      if (!(this_present_component_type && that_present_component_type))
+        return false;
+      if (!this.component_type.equals(that.component_type))
+        return false;
+    }
+
+    boolean this_present_topology_id = true && this.is_set_topology_id();
+    boolean that_present_topology_id = true && that.is_set_topology_id();
+    if (this_present_topology_id || that_present_topology_id) {
+      if (!(this_present_topology_id && that_present_topology_id))
+        return false;
+      if (!this.topology_id.equals(that.topology_id))
+        return false;
+    }
+
+    boolean this_present_topology_name = true && this.is_set_topology_name();
+    boolean that_present_topology_name = true && that.is_set_topology_name();
+    if (this_present_topology_name || that_present_topology_name) {
+      if (!(this_present_topology_name && that_present_topology_name))
+        return false;
+      if (!this.topology_name.equals(that.topology_name))
+        return false;
+    }
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_num_tasks = true && this.is_set_num_tasks();
+    boolean that_present_num_tasks = true && that.is_set_num_tasks();
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_window_to_stats = true && this.is_set_window_to_stats();
+    boolean that_present_window_to_stats = true && that.is_set_window_to_stats();
+    if (this_present_window_to_stats || that_present_window_to_stats) {
+      if (!(this_present_window_to_stats && that_present_window_to_stats))
+        return false;
+      if (!this.window_to_stats.equals(that.window_to_stats))
+        return false;
+    }
+
+    boolean this_present_gsid_to_input_stats = true && this.is_set_gsid_to_input_stats();
+    boolean that_present_gsid_to_input_stats = true && that.is_set_gsid_to_input_stats();
+    if (this_present_gsid_to_input_stats || that_present_gsid_to_input_stats) {
+      if (!(this_present_gsid_to_input_stats && that_present_gsid_to_input_stats))
+        return false;
+      if (!this.gsid_to_input_stats.equals(that.gsid_to_input_stats))
+        return false;
+    }
+
+    boolean this_present_sid_to_output_stats = true && this.is_set_sid_to_output_stats();
+    boolean that_present_sid_to_output_stats = true && that.is_set_sid_to_output_stats();
+    if (this_present_sid_to_output_stats || that_present_sid_to_output_stats) {
+      if (!(this_present_sid_to_output_stats && that_present_sid_to_output_stats))
+        return false;
+      if (!this.sid_to_output_stats.equals(that.sid_to_output_stats))
+        return false;
+    }
+
+    boolean this_present_exec_stats = true && this.is_set_exec_stats();
+    boolean that_present_exec_stats = true && that.is_set_exec_stats();
+    if (this_present_exec_stats || that_present_exec_stats) {
+      if (!(this_present_exec_stats && that_present_exec_stats))
+        return false;
+      if (!this.exec_stats.equals(that.exec_stats))
+        return false;
+    }
+
+    boolean this_present_errors = true && this.is_set_errors();
+    boolean that_present_errors = true && that.is_set_errors();
+    if (this_present_errors || that_present_errors) {
+      if (!(this_present_errors && that_present_errors))
+        return false;
+      if (!this.errors.equals(that.errors))
+        return false;
+    }
+
+    boolean this_present_eventlog_host = true && this.is_set_eventlog_host();
+    boolean that_present_eventlog_host = true && that.is_set_eventlog_host();
+    if (this_present_eventlog_host || that_present_eventlog_host) {
+      if (!(this_present_eventlog_host && that_present_eventlog_host))
+        return false;
+      if (!this.eventlog_host.equals(that.eventlog_host))
+        return false;
+    }
+
+    boolean this_present_eventlog_port = true && this.is_set_eventlog_port();
+    boolean that_present_eventlog_port = true && that.is_set_eventlog_port();
+    if (this_present_eventlog_port || that_present_eventlog_port) {
+      if (!(this_present_eventlog_port && that_present_eventlog_port))
+        return false;
+      if (this.eventlog_port != that.eventlog_port)
+        return false;
+    }
+
+    boolean this_present_debug_options = true && this.is_set_debug_options();
+    boolean that_present_debug_options = true && that.is_set_debug_options();
+    if (this_present_debug_options || that_present_debug_options) {
+      if (!(this_present_debug_options && that_present_debug_options))
+        return false;
+      if (!this.debug_options.equals(that.debug_options))
+        return false;
+    }
+
+    boolean this_present_topology_status = true && this.is_set_topology_status();
+    boolean that_present_topology_status = true && that.is_set_topology_status();
+    if (this_present_topology_status || that_present_topology_status) {
+      if (!(this_present_topology_status && that_present_topology_status))
+        return false;
+      if (!this.topology_status.equals(that.topology_status))
+        return false;
+    }
+
+    boolean this_present_resources_map = true && this.is_set_resources_map();
+    boolean that_present_resources_map = true && that.is_set_resources_map();
+    if (this_present_resources_map || that_present_resources_map) {
+      if (!(this_present_resources_map && that_present_resources_map))
+        return false;
+      if (!this.resources_map.equals(that.resources_map))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_component_id = true && (is_set_component_id());
+    list.add(present_component_id);
+    if (present_component_id)
+      list.add(component_id);
+
+    boolean present_component_type = true && (is_set_component_type());
+    list.add(present_component_type);
+    if (present_component_type)
+      list.add(component_type.getValue());
+
+    boolean present_topology_id = true && (is_set_topology_id());
+    list.add(present_topology_id);
+    if (present_topology_id)
+      list.add(topology_id);
+
+    boolean present_topology_name = true && (is_set_topology_name());
+    list.add(present_topology_name);
+    if (present_topology_name)
+      list.add(topology_name);
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_num_tasks = true && (is_set_num_tasks());
+    list.add(present_num_tasks);
+    if (present_num_tasks)
+      list.add(num_tasks);
+
+    boolean present_window_to_stats = true && (is_set_window_to_stats());
+    list.add(present_window_to_stats);
+    if (present_window_to_stats)
+      list.add(window_to_stats);
+
+    boolean present_gsid_to_input_stats = true && (is_set_gsid_to_input_stats());
+    list.add(present_gsid_to_input_stats);
+    if (present_gsid_to_input_stats)
+      list.add(gsid_to_input_stats);
+
+    boolean present_sid_to_output_stats = true && (is_set_sid_to_output_stats());
+    list.add(present_sid_to_output_stats);
+    if (present_sid_to_output_stats)
+      list.add(sid_to_output_stats);
+
+    boolean present_exec_stats = true && (is_set_exec_stats());
+    list.add(present_exec_stats);
+    if (present_exec_stats)
+      list.add(exec_stats);
+
+    boolean present_errors = true && (is_set_errors());
+    list.add(present_errors);
+    if (present_errors)
+      list.add(errors);
+
+    boolean present_eventlog_host = true && (is_set_eventlog_host());
+    list.add(present_eventlog_host);
+    if (present_eventlog_host)
+      list.add(eventlog_host);
+
+    boolean present_eventlog_port = true && (is_set_eventlog_port());
+    list.add(present_eventlog_port);
+    if (present_eventlog_port)
+      list.add(eventlog_port);
+
+    boolean present_debug_options = true && (is_set_debug_options());
+    list.add(present_debug_options);
+    if (present_debug_options)
+      list.add(debug_options);
+
+    boolean present_topology_status = true && (is_set_topology_status());
+    list.add(present_topology_status);
+    if (present_topology_status)
+      list.add(topology_status);
+
+    boolean present_resources_map = true && (is_set_resources_map());
+    list.add(present_resources_map);
+    if (present_resources_map)
+      list.add(resources_map);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ComponentPageInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(other.is_set_component_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, other.component_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_type()).compareTo(other.is_set_component_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_type, other.component_type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_name()).compareTo(other.is_set_topology_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_name, other.topology_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_stats()).compareTo(other.is_set_window_to_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_stats, other.window_to_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_gsid_to_input_stats()).compareTo(other.is_set_gsid_to_input_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_gsid_to_input_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.gsid_to_input_stats, other.gsid_to_input_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_sid_to_output_stats()).compareTo(other.is_set_sid_to_output_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sid_to_output_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sid_to_output_stats, other.sid_to_output_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_exec_stats()).compareTo(other.is_set_exec_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_exec_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exec_stats, other.exec_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_errors()).compareTo(other.is_set_errors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_errors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errors, other.errors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_eventlog_host()).compareTo(other.is_set_eventlog_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_eventlog_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eventlog_host, other.eventlog_host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_eventlog_port()).compareTo(other.is_set_eventlog_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_eventlog_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eventlog_port, other.eventlog_port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_debug_options()).compareTo(other.is_set_debug_options());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_debug_options()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug_options, other.debug_options);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_status()).compareTo(other.is_set_topology_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_status, other.topology_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_resources_map()).compareTo(other.is_set_resources_map());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_resources_map()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resources_map, other.resources_map);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ComponentPageInfo(");
+    boolean first = true;
+
+    sb.append("component_id:");
+    if (this.component_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.component_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("component_type:");
+    if (this.component_type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.component_type);
+    }
+    first = false;
+    if (is_set_topology_id()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_id:");
+      if (this.topology_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_id);
+      }
+      first = false;
+    }
+    if (is_set_topology_name()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_name:");
+      if (this.topology_name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_name);
+      }
+      first = false;
+    }
+    if (is_set_num_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("num_executors:");
+      sb.append(this.num_executors);
+      first = false;
+    }
+    if (is_set_num_tasks()) {
+      if (!first) sb.append(", ");
+      sb.append("num_tasks:");
+      sb.append(this.num_tasks);
+      first = false;
+    }
+    if (is_set_window_to_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_stats:");
+      if (this.window_to_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_stats);
+      }
+      first = false;
+    }
+    if (is_set_gsid_to_input_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("gsid_to_input_stats:");
+      if (this.gsid_to_input_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.gsid_to_input_stats);
+      }
+      first = false;
+    }
+    if (is_set_sid_to_output_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("sid_to_output_stats:");
+      if (this.sid_to_output_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sid_to_output_stats);
+      }
+      first = false;
+    }
+    if (is_set_exec_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("exec_stats:");
+      if (this.exec_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.exec_stats);
+      }
+      first = false;
+    }
+    if (is_set_errors()) {
+      if (!first) sb.append(", ");
+      sb.append("errors:");
+      if (this.errors == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.errors);
+      }
+      first = false;
+    }
+    if (is_set_eventlog_host()) {
+      if (!first) sb.append(", ");
+      sb.append("eventlog_host:");
+      if (this.eventlog_host == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.eventlog_host);
+      }
+      first = false;
+    }
+    if (is_set_eventlog_port()) {
+      if (!first) sb.append(", ");
+      sb.append("eventlog_port:");
+      sb.append(this.eventlog_port);
+      first = false;
+    }
+    if (is_set_debug_options()) {
+      if (!first) sb.append(", ");
+      sb.append("debug_options:");
+      if (this.debug_options == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.debug_options);
+      }
+      first = false;
+    }
+    if (is_set_topology_status()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_status:");
+      if (this.topology_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_status);
+      }
+      first = false;
+    }
+    if (is_set_resources_map()) {
+      if (!first) sb.append(", ");
+      sb.append("resources_map:");
+      if (this.resources_map == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.resources_map);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_component_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_component_type()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_type' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (debug_options != null) {
+      debug_options.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ComponentPageInfoStandardSchemeFactory implements SchemeFactory {
+    public ComponentPageInfoStandardScheme getScheme() {
+      return new ComponentPageInfoStandardScheme();
+    }
+  }
+
+  private static class ComponentPageInfoStandardScheme extends StandardScheme<ComponentPageInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // COMPONENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.component_id = iprot.readString();
+              struct.set_component_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMPONENT_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.component_type = org.apache.storm.generated.ComponentType.findByValue(iprot.readI32());
+              struct.set_component_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_id = iprot.readString();
+              struct.set_topology_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TOPOLOGY_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_name = iprot.readString();
+              struct.set_topology_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_tasks = iprot.readI32();
+              struct.set_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // WINDOW_TO_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map482 = iprot.readMapBegin();
+                struct.window_to_stats = new HashMap<String,ComponentAggregateStats>(2*_map482.size);
+                String _key483;
+                ComponentAggregateStats _val484;
+                for (int _i485 = 0; _i485 < _map482.size; ++_i485)
+                {
+                  _key483 = iprot.readString();
+                  _val484 = new ComponentAggregateStats();
+                  _val484.read(iprot);
+                  struct.window_to_stats.put(_key483, _val484);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // GSID_TO_INPUT_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map486 = iprot.readMapBegin();
+                struct.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(2*_map486.size);
+                GlobalStreamId _key487;
+                ComponentAggregateStats _val488;
+                for (int _i489 = 0; _i489 < _map486.size; ++_i489)
+                {
+                  _key487 = new GlobalStreamId();
+                  _key487.read(iprot);
+                  _val488 = new ComponentAggregateStats();
+                  _val488.read(iprot);
+                  struct.gsid_to_input_stats.put(_key487, _val488);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_gsid_to_input_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // SID_TO_OUTPUT_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map490 = iprot.readMapBegin();
+                struct.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(2*_map490.size);
+                String _key491;
+                ComponentAggregateStats _val492;
+                for (int _i493 = 0; _i493 < _map490.size; ++_i493)
+                {
+                  _key491 = iprot.readString();
+                  _val492 = new ComponentAggregateStats();
+                  _val492.read(iprot);
+                  struct.sid_to_output_stats.put(_key491, _val492);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_sid_to_output_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // EXEC_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list494 = iprot.readListBegin();
+                struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list494.size);
+                ExecutorAggregateStats _elem495;
+                for (int _i496 = 0; _i496 < _list494.size; ++_i496)
+                {
+                  _elem495 = new ExecutorAggregateStats();
+                  _elem495.read(iprot);
+                  struct.exec_stats.add(_elem495);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_exec_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 11: // ERRORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list497 = iprot.readListBegin();
+                struct.errors = new ArrayList<ErrorInfo>(_list497.size);
+                ErrorInfo _elem498;
+                for (int _i499 = 0; _i499 < _list497.size; ++_i499)
+                {
+                  _elem498 = new ErrorInfo();
+                  _elem498.read(iprot);
+                  struct.errors.add(_elem498);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_errors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // EVENTLOG_HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.eventlog_host = iprot.readString();
+              struct.set_eventlog_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // EVENTLOG_PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.eventlog_port = iprot.readI32();
+              struct.set_eventlog_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 14: // DEBUG_OPTIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.debug_options = new DebugOptions();
+              struct.debug_options.read(iprot);
+              struct.set_debug_options_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 15: // TOPOLOGY_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_status = iprot.readString();
+              struct.set_topology_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 16: // RESOURCES_MAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map500 = iprot.readMapBegin();
+                struct.resources_map = new HashMap<String,Double>(2*_map500.size);
+                String _key501;
+                double _val502;
+                for (int _i503 = 0; _i503 < _map500.size; ++_i503)
+                {
+                  _key501 = iprot.readString();
+                  _val502 = iprot.readDouble();
+                  struct.resources_map.put(_key501, _val502);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_resources_map_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.component_id != null) {
+        oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+        oprot.writeString(struct.component_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.component_type != null) {
+        oprot.writeFieldBegin(COMPONENT_TYPE_FIELD_DESC);
+        oprot.writeI32(struct.component_type.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.topology_id != null) {
+        if (struct.is_set_topology_id()) {
+          oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+          oprot.writeString(struct.topology_id);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_name != null) {
+        if (struct.is_set_topology_name()) {
+          oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC);
+          oprot.writeString(struct.topology_name);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+        oprot.writeI32(struct.num_executors);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+        oprot.writeI32(struct.num_tasks);
+        oprot.writeFieldEnd();
+      }
+      if (struct.window_to_stats != null) {
+        if (struct.is_set_window_to_stats()) {
+          oprot.writeFieldBegin(WINDOW_TO_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.window_to_stats.size()));
+            for (Map.Entry<String, ComponentAggregateStats> _iter504 : struct.window_to_stats.entrySet())
+            {
+              oprot.writeString(_iter504.getKey());
+              _iter504.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.gsid_to_input_stats != null) {
+        if (struct.is_set_gsid_to_input_stats()) {
+          oprot.writeFieldBegin(GSID_TO_INPUT_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.gsid_to_input_stats.size()));
+            for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter505 : struct.gsid_to_input_stats.entrySet())
+            {
+              _iter505.getKey().write(oprot);
+              _iter505.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.sid_to_output_stats != null) {
+        if (struct.is_set_sid_to_output_stats()) {
+          oprot.writeFieldBegin(SID_TO_OUTPUT_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.sid_to_output_stats.size()));
+            for (Map.Entry<String, ComponentAggregateStats> _iter506 : struct.sid_to_output_stats.entrySet())
+            {
+              oprot.writeString(_iter506.getKey());
+              _iter506.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.exec_stats != null) {
+        if (struct.is_set_exec_stats()) {
+          oprot.writeFieldBegin(EXEC_STATS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.exec_stats.size()));
+            for (ExecutorAggregateStats _iter507 : struct.exec_stats)
+            {
+              _iter507.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.errors != null) {
+        if (struct.is_set_errors()) {
+          oprot.writeFieldBegin(ERRORS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.errors.size()));
+            for (ErrorInfo _iter508 : struct.errors)
+            {
+              _iter508.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.eventlog_host != null) {
+        if (struct.is_set_eventlog_host()) {
+          oprot.writeFieldBegin(EVENTLOG_HOST_FIELD_DESC);
+          oprot.writeString(struct.eventlog_host);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_eventlog_port()) {
+        oprot.writeFieldBegin(EVENTLOG_PORT_FIELD_DESC);
+        oprot.writeI32(struct.eventlog_port);
+        oprot.writeFieldEnd();
+      }
+      if (struct.debug_options != null) {
+        if (struct.is_set_debug_options()) {
+          oprot.writeFieldBegin(DEBUG_OPTIONS_FIELD_DESC);
+          struct.debug_options.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_status != null) {
+        if (struct.is_set_topology_status()) {
+          oprot.writeFieldBegin(TOPOLOGY_STATUS_FIELD_DESC);
+          oprot.writeString(struct.topology_status);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.resources_map != null) {
+        if (struct.is_set_resources_map()) {
+          oprot.writeFieldBegin(RESOURCES_MAP_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.resources_map.size()));
+            for (Map.Entry<String, Double> _iter509 : struct.resources_map.entrySet())
+            {
+              oprot.writeString(_iter509.getKey());
+              oprot.writeDouble(_iter509.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ComponentPageInfoTupleSchemeFactory implements SchemeFactory {
+    public ComponentPageInfoTupleScheme getScheme() {
+      return new ComponentPageInfoTupleScheme();
+    }
+  }
+
+  private static class ComponentPageInfoTupleScheme extends TupleScheme<ComponentPageInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.component_id);
+      oprot.writeI32(struct.component_type.getValue());
+      BitSet optionals = new BitSet();
+      if (struct.is_set_topology_id()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_topology_name()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_num_executors()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_num_tasks()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_window_to_stats()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_gsid_to_input_stats()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_sid_to_output_stats()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_exec_stats()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_errors()) {
+        optionals.set(8);
+      }
+      if (struct.is_set_eventlog_host()) {
+        optionals.set(9);
+      }
+      if (struct.is_set_eventlog_port()) {
+        optionals.set(10);
+      }
+      if (struct.is_set_debug_options()) {
+        optionals.set(11);
+      }
+      if (struct.is_set_topology_status()) {
+        optionals.set(12);
+      }
+      if (struct.is_set_resources_map()) {
+        optionals.set(13);
+      }
+      oprot.writeBitSet(optionals, 14);
+      if (struct.is_set_topology_id()) {
+        oprot.writeString(struct.topology_id);
+      }
+      if (struct.is_set_topology_name()) {
+        oprot.writeString(struct.topology_name);
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeI32(struct.num_executors);
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeI32(struct.num_tasks);
+      }
+      if (struct.is_set_window_to_stats()) {
+        {
+          oprot.writeI32(struct.window_to_stats.size());
+          for (Map.Entry<String, ComponentAggregateStats> _iter510 : struct.window_to_stats.entrySet())
+          {
+            oprot.writeString(_iter510.getKey());
+            _iter510.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_gsid_to_input_stats()) {
+        {
+          oprot.writeI32(struct.gsid_to_input_stats.size());
+          for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter511 : struct.gsid_to_input_stats.entrySet())
+          {
+            _iter511.getKey().write(oprot);
+            _iter511.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_sid_to_output_stats()) {
+        {
+          oprot.writeI32(struct.sid_to_output_stats.size());
+          for (Map.Entry<String, ComponentAggregateStats> _iter512 : struct.sid_to_output_stats.entrySet())
+          {
+            oprot.writeString(_iter512.getKey());
+            _iter512.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_exec_stats()) {
+        {
+          oprot.writeI32(struct.exec_stats.size());
+          for (ExecutorAggregateStats _iter513 : struct.exec_stats)
+          {
+            _iter513.write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_errors()) {
+        {
+          oprot.writeI32(struct.errors.size());
+          for (ErrorInfo _iter514 : struct.errors)
+          {
+            _iter514.write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_eventlog_host()) {
+        oprot.writeString(struct.eventlog_host);
+      }
+      if (struct.is_set_eventlog_port()) {
+        oprot.writeI32(struct.eventlog_port);
+      }
+      if (struct.is_set_debug_options()) {
+        struct.debug_options.write(oprot);
+      }
+      if (struct.is_set_topology_status()) {
+        oprot.writeString(struct.topology_status);
+      }
+      if (struct.is_set_resources_map()) {
+        {
+          oprot.writeI32(struct.resources_map.size());
+          for (Map.Entry<String, Double> _iter515 : struct.resources_map.entrySet())
+          {
+            oprot.writeString(_iter515.getKey());
+            oprot.writeDouble(_iter515.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.component_id = iprot.readString();
+      struct.set_component_id_isSet(true);
+      struct.component_type = org.apache.storm.generated.ComponentType.findByValue(iprot.readI32());
+      struct.set_component_type_isSet(true);
+      BitSet incoming = iprot.readBitSet(14);
+      if (incoming.get(0)) {
+        struct.topology_id = iprot.readString();
+        struct.set_topology_id_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.topology_name = iprot.readString();
+        struct.set_topology_name_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.num_executors = iprot.readI32();
+        struct.set_num_executors_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.num_tasks = iprot.readI32();
+        struct.set_num_tasks_isSet(true);
+      }
+      if (incoming.get(4)) {
+        {
+          org.apache.thrift.protocol.TMap _map516 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.window_to_stats = new HashMap<String,ComponentAggregateStats>(2*_map516.size);
+          String _key517;
+          ComponentAggregateStats _val518;
+          for (int _i519 = 0; _i519 < _map516.size; ++_i519)
+          {
+            _key517 = iprot.readString();
+            _val518 = new ComponentAggregateStats();
+            _val518.read(iprot);
+            struct.window_to_stats.put(_key517, _val518);
+          }
+        }
+        struct.set_window_to_stats_isSet(true);
+      }
+      if (incoming.get(5)) {
+        {
+          org.apache.thrift.protocol.TMap _map520 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(2*_map520.size);
+          GlobalStreamId _key521;
+          ComponentAggregateStats _val522;
+          for (int _i523 = 0; _i523 < _map520.size; ++_i523)
+          {
+            _key521 = new GlobalStreamId();
+            _key521.read(iprot);
+            _val522 = new ComponentAggregateStats();
+            _val522.read(iprot);
+            struct.gsid_to_input_stats.put(_key521, _val522);
+          }
+        }
+        struct.set_gsid_to_input_stats_isSet(true);
+      }
+      if (incoming.get(6)) {
+        {
+          org.apache.thrift.protocol.TMap _map524 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(2*_map524.size);
+          String _key525;
+          ComponentAggregateStats _val526;
+          for (int _i527 = 0; _i527 < _map524.size; ++_i527)
+          {
+            _key525 = iprot.readString();
+            _val526 = new ComponentAggregateStats();
+            _val526.read(iprot);
+            struct.sid_to_output_stats.put(_key525, _val526);
+          }
+        }
+        struct.set_sid_to_output_stats_isSet(true);
+      }
+      if (incoming.get(7)) {
+        {
+          org.apache.thrift.protocol.TList _list528 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list528.size);
+          ExecutorAggregateStats _elem529;
+          for (int _i530 = 0; _i530 < _list528.size; ++_i530)
+          {
+            _elem529 = new ExecutorAggregateStats();
+            _elem529.read(iprot);
+            struct.exec_stats.add(_elem529);
+          }
+        }
+        struct.set_exec_stats_isSet(true);
+      }
+      if (incoming.get(8)) {
+        {
+          org.apache.thrift.protocol.TList _list531 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.errors = new ArrayList<ErrorInfo>(_list531.size);
+          ErrorInfo _elem532;
+          for (int _i533 = 0; _i533 < _list531.size; ++_i533)
+          {
+            _elem532 = new ErrorInfo();
+            _elem532.read(iprot);
+            struct.errors.add(_elem532);
+          }
+        }
+        struct.set_errors_isSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.eventlog_host = iprot.readString();
+        struct.set_eventlog_host_isSet(true);
+      }
+      if (incoming.get(10)) {
+        struct.eventlog_port = iprot.readI32();
+        struct.set_eventlog_port_isSet(true);
+      }
+      if (incoming.get(11)) {
+        struct.debug_options = new DebugOptions();
+        struct.debug_options.read(iprot);
+        struct.set_debug_options_isSet(true);
+      }
+      if (incoming.get(12)) {
+        struct.topology_status = iprot.readString();
+        struct.set_topology_status_isSet(true);
+      }
+      if (incoming.get(13)) {
+        {
+          org.apache.thrift.protocol.TMap _map534 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.resources_map = new HashMap<String,Double>(2*_map534.size);
+          String _key535;
+          double _val536;
+          for (int _i537 = 0; _i537 < _map534.size; ++_i537)
+          {
+            _key535 = iprot.readString();
+            _val536 = iprot.readDouble();
+            struct.resources_map.put(_key535, _val536);
+          }
+        }
+        stru

<TRUNCATED>

[18/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/NimbusSummary.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/NimbusSummary.java b/storm-client/src/jvm/org/apache/storm/generated/NimbusSummary.java
new file mode 100644
index 0000000..c0c5247
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/NimbusSummary.java
@@ -0,0 +1,796 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, NimbusSummary._Fields>, java.io.Serializable, Cloneable, Comparable<NimbusSummary> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusSummary");
+
+  private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField IS_LEADER_FIELD_DESC = new org.apache.thrift.protocol.TField("isLeader", org.apache.thrift.protocol.TType.BOOL, (short)4);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NimbusSummaryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NimbusSummaryTupleSchemeFactory());
+  }
+
+  private String host; // required
+  private int port; // required
+  private int uptime_secs; // required
+  private boolean isLeader; // required
+  private String version; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    HOST((short)1, "host"),
+    PORT((short)2, "port"),
+    UPTIME_SECS((short)3, "uptime_secs"),
+    IS_LEADER((short)4, "isLeader"),
+    VERSION((short)5, "version");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // HOST
+          return HOST;
+        case 2: // PORT
+          return PORT;
+        case 3: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 4: // IS_LEADER
+          return IS_LEADER;
+        case 5: // VERSION
+          return VERSION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PORT_ISSET_ID = 0;
+  private static final int __UPTIME_SECS_ISSET_ID = 1;
+  private static final int __ISLEADER_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.IS_LEADER, new org.apache.thrift.meta_data.FieldMetaData("isLeader", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NimbusSummary.class, metaDataMap);
+  }
+
+  public NimbusSummary() {
+  }
+
+  public NimbusSummary(
+    String host,
+    int port,
+    int uptime_secs,
+    boolean isLeader,
+    String version)
+  {
+    this();
+    this.host = host;
+    this.port = port;
+    set_port_isSet(true);
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+    this.isLeader = isLeader;
+    set_isLeader_isSet(true);
+    this.version = version;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NimbusSummary(NimbusSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.port = other.port;
+    this.uptime_secs = other.uptime_secs;
+    this.isLeader = other.isLeader;
+    if (other.is_set_version()) {
+      this.version = other.version;
+    }
+  }
+
+  public NimbusSummary deepCopy() {
+    return new NimbusSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.host = null;
+    set_port_isSet(false);
+    this.port = 0;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    set_isLeader_isSet(false);
+    this.isLeader = false;
+    this.version = null;
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public boolean is_isLeader() {
+    return this.isLeader;
+  }
+
+  public void set_isLeader(boolean isLeader) {
+    this.isLeader = isLeader;
+    set_isLeader_isSet(true);
+  }
+
+  public void unset_isLeader() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISLEADER_ISSET_ID);
+  }
+
+  /** Returns true if field isLeader is set (has been assigned a value) and false otherwise */
+  public boolean is_set_isLeader() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISLEADER_ISSET_ID);
+  }
+
+  public void set_isLeader_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISLEADER_ISSET_ID, value);
+  }
+
+  public String get_version() {
+    return this.version;
+  }
+
+  public void set_version(String version) {
+    this.version = version;
+  }
+
+  public void unset_version() {
+    this.version = null;
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return this.version != null;
+  }
+
+  public void set_version_isSet(boolean value) {
+    if (!value) {
+      this.version = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case IS_LEADER:
+      if (value == null) {
+        unset_isLeader();
+      } else {
+        set_isLeader((Boolean)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case HOST:
+      return get_host();
+
+    case PORT:
+      return get_port();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case IS_LEADER:
+      return is_isLeader();
+
+    case VERSION:
+      return get_version();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case HOST:
+      return is_set_host();
+    case PORT:
+      return is_set_port();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case IS_LEADER:
+      return is_set_isLeader();
+    case VERSION:
+      return is_set_version();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NimbusSummary)
+      return this.equals((NimbusSummary)that);
+    return false;
+  }
+
+  public boolean equals(NimbusSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_port = true;
+    boolean that_present_port = true;
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_isLeader = true;
+    boolean that_present_isLeader = true;
+    if (this_present_isLeader || that_present_isLeader) {
+      if (!(this_present_isLeader && that_present_isLeader))
+        return false;
+      if (this.isLeader != that.isLeader)
+        return false;
+    }
+
+    boolean this_present_version = true && this.is_set_version();
+    boolean that_present_version = true && that.is_set_version();
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (!this.version.equals(that.version))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_host = true && (is_set_host());
+    list.add(present_host);
+    if (present_host)
+      list.add(host);
+
+    boolean present_port = true;
+    list.add(present_port);
+    if (present_port)
+      list.add(port);
+
+    boolean present_uptime_secs = true;
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_isLeader = true;
+    list.add(present_isLeader);
+    if (present_isLeader)
+      list.add(isLeader);
+
+    boolean present_version = true && (is_set_version());
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(NimbusSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_isLeader()).compareTo(other.is_set_isLeader());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_isLeader()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isLeader, other.isLeader);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NimbusSummary(");
+    boolean first = true;
+
+    sb.append("host:");
+    if (this.host == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.host);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    sb.append(this.port);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("isLeader:");
+    sb.append(this.isLeader);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("version:");
+    if (this.version == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.version);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_host()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_isLeader()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'isLeader' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class NimbusSummaryStandardSchemeFactory implements SchemeFactory {
+    public NimbusSummaryStandardScheme getScheme() {
+      return new NimbusSummaryStandardScheme();
+    }
+  }
+
+  private static class NimbusSummaryStandardScheme extends StandardScheme<NimbusSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NimbusSummary struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.host = iprot.readString();
+              struct.set_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.port = iprot.readI32();
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // IS_LEADER
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isLeader = iprot.readBool();
+              struct.set_isLeader_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.version = iprot.readString();
+              struct.set_version_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, NimbusSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.host != null) {
+        oprot.writeFieldBegin(HOST_FIELD_DESC);
+        oprot.writeString(struct.host);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(PORT_FIELD_DESC);
+      oprot.writeI32(struct.port);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(IS_LEADER_FIELD_DESC);
+      oprot.writeBool(struct.isLeader);
+      oprot.writeFieldEnd();
+      if (struct.version != null) {
+        oprot.writeFieldBegin(VERSION_FIELD_DESC);
+        oprot.writeString(struct.version);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NimbusSummaryTupleSchemeFactory implements SchemeFactory {
+    public NimbusSummaryTupleScheme getScheme() {
+      return new NimbusSummaryTupleScheme();
+    }
+  }
+
+  private static class NimbusSummaryTupleScheme extends TupleScheme<NimbusSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NimbusSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.host);
+      oprot.writeI32(struct.port);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeBool(struct.isLeader);
+      oprot.writeString(struct.version);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NimbusSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.host = iprot.readString();
+      struct.set_host_isSet(true);
+      struct.port = iprot.readI32();
+      struct.set_port_isSet(true);
+      struct.uptime_secs = iprot.readI32();
+      struct.set_uptime_secs_isSet(true);
+      struct.isLeader = iprot.readBool();
+      struct.set_isLeader_isSet(true);
+      struct.version = iprot.readString();
+      struct.set_version_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java b/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java
new file mode 100644
index 0000000..5776717
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java
@@ -0,0 +1,556 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<NodeInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo");
+
+  private static final org.apache.thrift.protocol.TField NODE_FIELD_DESC = new org.apache.thrift.protocol.TField("node", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.SET, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NodeInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NodeInfoTupleSchemeFactory());
+  }
+
+  private String node; // required
+  private Set<Long> port; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NODE((short)1, "node"),
+    PORT((short)2, "port");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NODE
+          return NODE;
+        case 2: // PORT
+          return PORT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NODE, new org.apache.thrift.meta_data.FieldMetaData("node", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NodeInfo.class, metaDataMap);
+  }
+
+  public NodeInfo() {
+  }
+
+  public NodeInfo(
+    String node,
+    Set<Long> port)
+  {
+    this();
+    this.node = node;
+    this.port = port;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NodeInfo(NodeInfo other) {
+    if (other.is_set_node()) {
+      this.node = other.node;
+    }
+    if (other.is_set_port()) {
+      Set<Long> __this__port = new HashSet<Long>(other.port);
+      this.port = __this__port;
+    }
+  }
+
+  public NodeInfo deepCopy() {
+    return new NodeInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.node = null;
+    this.port = null;
+  }
+
+  public String get_node() {
+    return this.node;
+  }
+
+  public void set_node(String node) {
+    this.node = node;
+  }
+
+  public void unset_node() {
+    this.node = null;
+  }
+
+  /** Returns true if field node is set (has been assigned a value) and false otherwise */
+  public boolean is_set_node() {
+    return this.node != null;
+  }
+
+  public void set_node_isSet(boolean value) {
+    if (!value) {
+      this.node = null;
+    }
+  }
+
+  public int get_port_size() {
+    return (this.port == null) ? 0 : this.port.size();
+  }
+
+  public java.util.Iterator<Long> get_port_iterator() {
+    return (this.port == null) ? null : this.port.iterator();
+  }
+
+  public void add_to_port(long elem) {
+    if (this.port == null) {
+      this.port = new HashSet<Long>();
+    }
+    this.port.add(elem);
+  }
+
+  public Set<Long> get_port() {
+    return this.port;
+  }
+
+  public void set_port(Set<Long> port) {
+    this.port = port;
+  }
+
+  public void unset_port() {
+    this.port = null;
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return this.port != null;
+  }
+
+  public void set_port_isSet(boolean value) {
+    if (!value) {
+      this.port = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NODE:
+      if (value == null) {
+        unset_node();
+      } else {
+        set_node((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Set<Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NODE:
+      return get_node();
+
+    case PORT:
+      return get_port();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NODE:
+      return is_set_node();
+    case PORT:
+      return is_set_port();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NodeInfo)
+      return this.equals((NodeInfo)that);
+    return false;
+  }
+
+  public boolean equals(NodeInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_node = true && this.is_set_node();
+    boolean that_present_node = true && that.is_set_node();
+    if (this_present_node || that_present_node) {
+      if (!(this_present_node && that_present_node))
+        return false;
+      if (!this.node.equals(that.node))
+        return false;
+    }
+
+    boolean this_present_port = true && this.is_set_port();
+    boolean that_present_port = true && that.is_set_port();
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (!this.port.equals(that.port))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_node = true && (is_set_node());
+    list.add(present_node);
+    if (present_node)
+      list.add(node);
+
+    boolean present_port = true && (is_set_port());
+    list.add(present_port);
+    if (present_port)
+      list.add(port);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(NodeInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_node()).compareTo(other.is_set_node());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_node()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node, other.node);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NodeInfo(");
+    boolean first = true;
+
+    sb.append("node:");
+    if (this.node == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.node);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    if (this.port == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.port);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_node()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'node' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class NodeInfoStandardSchemeFactory implements SchemeFactory {
+    public NodeInfoStandardScheme getScheme() {
+      return new NodeInfoStandardScheme();
+    }
+  }
+
+  private static class NodeInfoStandardScheme extends StandardScheme<NodeInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NodeInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NODE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.node = iprot.readString();
+              struct.set_node_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
+              {
+                org.apache.thrift.protocol.TSet _set610 = iprot.readSetBegin();
+                struct.port = new HashSet<Long>(2*_set610.size);
+                long _elem611;
+                for (int _i612 = 0; _i612 < _set610.size; ++_i612)
+                {
+                  _elem611 = iprot.readI64();
+                  struct.port.add(_elem611);
+                }
+                iprot.readSetEnd();
+              }
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, NodeInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.node != null) {
+        oprot.writeFieldBegin(NODE_FIELD_DESC);
+        oprot.writeString(struct.node);
+        oprot.writeFieldEnd();
+      }
+      if (struct.port != null) {
+        oprot.writeFieldBegin(PORT_FIELD_DESC);
+        {
+          oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size()));
+          for (long _iter613 : struct.port)
+          {
+            oprot.writeI64(_iter613);
+          }
+          oprot.writeSetEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NodeInfoTupleSchemeFactory implements SchemeFactory {
+    public NodeInfoTupleScheme getScheme() {
+      return new NodeInfoTupleScheme();
+    }
+  }
+
+  private static class NodeInfoTupleScheme extends TupleScheme<NodeInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.node);
+      {
+        oprot.writeI32(struct.port.size());
+        for (long _iter614 : struct.port)
+        {
+          oprot.writeI64(_iter614);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.node = iprot.readString();
+      struct.set_node_isSet(true);
+      {
+        org.apache.thrift.protocol.TSet _set615 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.port = new HashSet<Long>(2*_set615.size);
+        long _elem616;
+        for (int _i617 = 0; _i617 < _set615.size; ++_i617)
+        {
+          _elem616 = iprot.readI64();
+          struct.port.add(_elem616);
+        }
+      }
+      struct.set_port_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/NotAliveException.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/NotAliveException.java b/storm-client/src/jvm/org/apache/storm/generated/NotAliveException.java
new file mode 100644
index 0000000..8d9374a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/NotAliveException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class NotAliveException extends TException implements org.apache.thrift.TBase<NotAliveException, NotAliveException._Fields>, java.io.Serializable, Cloneable, Comparable<NotAliveException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotAliveException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NotAliveExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NotAliveExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NotAliveException.class, metaDataMap);
+  }
+
+  public NotAliveException() {
+  }
+
+  public NotAliveException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NotAliveException(NotAliveException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public NotAliveException deepCopy() {
+    return new NotAliveException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NotAliveException)
+      return this.equals((NotAliveException)that);
+    return false;
+  }
+
+  public boolean equals(NotAliveException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(NotAliveException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NotAliveException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class NotAliveExceptionStandardSchemeFactory implements SchemeFactory {
+    public NotAliveExceptionStandardScheme getScheme() {
+      return new NotAliveExceptionStandardScheme();
+    }
+  }
+
+  private static class NotAliveExceptionStandardScheme extends StandardScheme<NotAliveException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NotAliveException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, NotAliveException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NotAliveExceptionTupleSchemeFactory implements SchemeFactory {
+    public NotAliveExceptionTupleScheme getScheme() {
+      return new NotAliveExceptionTupleScheme();
+    }
+  }
+
+  private static class NotAliveExceptionTupleScheme extends TupleScheme<NotAliveException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NotAliveException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NotAliveException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/NullStruct.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/NullStruct.java b/storm-client/src/jvm/org/apache/storm/generated/NullStruct.java
new file mode 100644
index 0000000..a37ecbd
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/NullStruct.java
@@ -0,0 +1,300 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class NullStruct implements org.apache.thrift.TBase<NullStruct, NullStruct._Fields>, java.io.Serializable, Cloneable, Comparable<NullStruct> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NullStruct");
+
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NullStructStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NullStructTupleSchemeFactory());
+  }
+
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NullStruct.class, metaDataMap);
+  }
+
+  public NullStruct() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NullStruct(NullStruct other) {
+  }
+
+  public NullStruct deepCopy() {
+    return new NullStruct(this);
+  }
+
+  @Override
+  public void clear() {
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NullStruct)
+      return this.equals((NullStruct)that);
+    return false;
+  }
+
+  public boolean equals(NullStruct that) {
+    if (that == null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(NullStruct other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NullStruct(");
+    boolean first = true;
+
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class NullStructStandardSchemeFactory implements SchemeFactory {
+    public NullStructStandardScheme getScheme() {
+      return new NullStructStandardScheme();
+    }
+  }
+
+  private static class NullStructStandardScheme extends StandardScheme<NullStruct> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NullStruct struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, NullStruct struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NullStructTupleSchemeFactory implements SchemeFactory {
+    public NullStructTupleScheme getScheme() {
+      return new NullStructTupleScheme();
+    }
+  }
+
+  private static class NullStructTupleScheme extends TupleScheme<NullStruct> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NullStruct struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NullStruct struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/NumErrorsChoice.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/NumErrorsChoice.java b/storm-client/src/jvm/org/apache/storm/generated/NumErrorsChoice.java
new file mode 100644
index 0000000..73421ee
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/NumErrorsChoice.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum NumErrorsChoice implements org.apache.thrift.TEnum {
+  ALL(0),
+  NONE(1),
+  ONE(2);
+
+  private final int value;
+
+  private NumErrorsChoice(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static NumErrorsChoice findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return ALL;
+      case 1:
+        return NONE;
+      case 2:
+        return ONE;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ProfileAction.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ProfileAction.java b/storm-client/src/jvm/org/apache/storm/generated/ProfileAction.java
new file mode 100644
index 0000000..f783c98
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ProfileAction.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum ProfileAction implements org.apache.thrift.TEnum {
+  JPROFILE_STOP(0),
+  JPROFILE_START(1),
+  JPROFILE_DUMP(2),
+  JMAP_DUMP(3),
+  JSTACK_DUMP(4),
+  JVM_RESTART(5);
+
+  private final int value;
+
+  private ProfileAction(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static ProfileAction findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return JPROFILE_STOP;
+      case 1:
+        return JPROFILE_START;
+      case 2:
+        return JPROFILE_DUMP;
+      case 3:
+        return JMAP_DUMP;
+      case 4:
+        return JSTACK_DUMP;
+      case 5:
+        return JVM_RESTART;
+      default:
+        return null;
+    }
+  }
+}


[29/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/DistributedRPCInvocations.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/DistributedRPCInvocations.java b/storm-client/src/jvm/org/apache/storm/generated/DistributedRPCInvocations.java
new file mode 100644
index 0000000..7d1c1ab
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/DistributedRPCInvocations.java
@@ -0,0 +1,3904 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class DistributedRPCInvocations {
+
+  public interface Iface {
+
+    public void result(String id, String result) throws AuthorizationException, org.apache.thrift.TException;
+
+    public DRPCRequest fetchRequest(String functionName) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void failRequest(String id) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void failRequestV2(String id, DRPCExecutionException e) throws AuthorizationException, org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void result(String id, String result, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void fetchRequest(String functionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void failRequest(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void failRequestV2(String id, DRPCExecutionException e, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    public void result(String id, String result) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_result(id, result);
+      recv_result();
+    }
+
+    public void send_result(String id, String result) throws org.apache.thrift.TException
+    {
+      result_args args = new result_args();
+      args.set_id(id);
+      args.set_result(result);
+      sendBase("result", args);
+    }
+
+    public void recv_result() throws AuthorizationException, org.apache.thrift.TException
+    {
+      result_result result = new result_result();
+      receiveBase(result, "result");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public DRPCRequest fetchRequest(String functionName) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_fetchRequest(functionName);
+      return recv_fetchRequest();
+    }
+
+    public void send_fetchRequest(String functionName) throws org.apache.thrift.TException
+    {
+      fetchRequest_args args = new fetchRequest_args();
+      args.set_functionName(functionName);
+      sendBase("fetchRequest", args);
+    }
+
+    public DRPCRequest recv_fetchRequest() throws AuthorizationException, org.apache.thrift.TException
+    {
+      fetchRequest_result result = new fetchRequest_result();
+      receiveBase(result, "fetchRequest");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result");
+    }
+
+    public void failRequest(String id) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_failRequest(id);
+      recv_failRequest();
+    }
+
+    public void send_failRequest(String id) throws org.apache.thrift.TException
+    {
+      failRequest_args args = new failRequest_args();
+      args.set_id(id);
+      sendBase("failRequest", args);
+    }
+
+    public void recv_failRequest() throws AuthorizationException, org.apache.thrift.TException
+    {
+      failRequest_result result = new failRequest_result();
+      receiveBase(result, "failRequest");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void failRequestV2(String id, DRPCExecutionException e) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_failRequestV2(id, e);
+      recv_failRequestV2();
+    }
+
+    public void send_failRequestV2(String id, DRPCExecutionException e) throws org.apache.thrift.TException
+    {
+      failRequestV2_args args = new failRequestV2_args();
+      args.set_id(id);
+      args.set_e(e);
+      sendBase("failRequestV2", args);
+    }
+
+    public void recv_failRequestV2() throws AuthorizationException, org.apache.thrift.TException
+    {
+      failRequestV2_result result = new failRequestV2_result();
+      receiveBase(result, "failRequestV2");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void result(String id, String result, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      result_call method_call = new result_call(id, result, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class result_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      private String result;
+      public result_call(String id, String result, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.result = result;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("result", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        result_args args = new result_args();
+        args.set_id(id);
+        args.set_result(result);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_result();
+      }
+    }
+
+    public void fetchRequest(String functionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      fetchRequest_call method_call = new fetchRequest_call(functionName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class fetchRequest_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String functionName;
+      public fetchRequest_call(String functionName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.functionName = functionName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("fetchRequest", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        fetchRequest_args args = new fetchRequest_args();
+        args.set_functionName(functionName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public DRPCRequest getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_fetchRequest();
+      }
+    }
+
+    public void failRequest(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      failRequest_call method_call = new failRequest_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class failRequest_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      public failRequest_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("failRequest", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        failRequest_args args = new failRequest_args();
+        args.set_id(id);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_failRequest();
+      }
+    }
+
+    public void failRequestV2(String id, DRPCExecutionException e, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      failRequestV2_call method_call = new failRequestV2_call(id, e, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class failRequestV2_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      private DRPCExecutionException e;
+      public failRequestV2_call(String id, DRPCExecutionException e, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.e = e;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("failRequestV2", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        failRequestV2_args args = new failRequestV2_args();
+        args.set_id(id);
+        args.set_e(e);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_failRequestV2();
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("result", new result());
+      processMap.put("fetchRequest", new fetchRequest());
+      processMap.put("failRequest", new failRequest());
+      processMap.put("failRequestV2", new failRequestV2());
+      return processMap;
+    }
+
+    public static class result<I extends Iface> extends org.apache.thrift.ProcessFunction<I, result_args> {
+      public result() {
+        super("result");
+      }
+
+      public result_args getEmptyArgsInstance() {
+        return new result_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public result_result getResult(I iface, result_args args) throws org.apache.thrift.TException {
+        result_result result = new result_result();
+        try {
+          iface.result(args.id, args.result);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
+    public static class fetchRequest<I extends Iface> extends org.apache.thrift.ProcessFunction<I, fetchRequest_args> {
+      public fetchRequest() {
+        super("fetchRequest");
+      }
+
+      public fetchRequest_args getEmptyArgsInstance() {
+        return new fetchRequest_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public fetchRequest_result getResult(I iface, fetchRequest_args args) throws org.apache.thrift.TException {
+        fetchRequest_result result = new fetchRequest_result();
+        try {
+          result.success = iface.fetchRequest(args.functionName);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
+    public static class failRequest<I extends Iface> extends org.apache.thrift.ProcessFunction<I, failRequest_args> {
+      public failRequest() {
+        super("failRequest");
+      }
+
+      public failRequest_args getEmptyArgsInstance() {
+        return new failRequest_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public failRequest_result getResult(I iface, failRequest_args args) throws org.apache.thrift.TException {
+        failRequest_result result = new failRequest_result();
+        try {
+          iface.failRequest(args.id);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
+    public static class failRequestV2<I extends Iface> extends org.apache.thrift.ProcessFunction<I, failRequestV2_args> {
+      public failRequestV2() {
+        super("failRequestV2");
+      }
+
+      public failRequestV2_args getEmptyArgsInstance() {
+        return new failRequestV2_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public failRequestV2_result getResult(I iface, failRequestV2_args args) throws org.apache.thrift.TException {
+        failRequestV2_result result = new failRequestV2_result();
+        try {
+          iface.failRequestV2(args.id, args.e);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
+  }
+
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName());
+    public AsyncProcessor(I iface) {
+      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+    }
+
+    protected AsyncProcessor(I iface, Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends AsyncIface> Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      processMap.put("result", new result());
+      processMap.put("fetchRequest", new fetchRequest());
+      processMap.put("failRequest", new failRequest());
+      processMap.put("failRequestV2", new failRequestV2());
+      return processMap;
+    }
+
+    public static class result<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, result_args, Void> {
+      public result() {
+        super("result");
+      }
+
+      public result_args getEmptyArgsInstance() {
+        return new result_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            result_result result = new result_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            result_result result = new result_result();
+            if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, result_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.result(args.id, args.result,resultHandler);
+      }
+    }
+
+    public static class fetchRequest<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fetchRequest_args, DRPCRequest> {
+      public fetchRequest() {
+        super("fetchRequest");
+      }
+
+      public fetchRequest_args getEmptyArgsInstance() {
+        return new fetchRequest_args();
+      }
+
+      public AsyncMethodCallback<DRPCRequest> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<DRPCRequest>() { 
+          public void onComplete(DRPCRequest o) {
+            fetchRequest_result result = new fetchRequest_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            fetchRequest_result result = new fetchRequest_result();
+            if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, fetchRequest_args args, org.apache.thrift.async.AsyncMethodCallback<DRPCRequest> resultHandler) throws TException {
+        iface.fetchRequest(args.functionName,resultHandler);
+      }
+    }
+
+    public static class failRequest<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, failRequest_args, Void> {
+      public failRequest() {
+        super("failRequest");
+      }
+
+      public failRequest_args getEmptyArgsInstance() {
+        return new failRequest_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            failRequest_result result = new failRequest_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            failRequest_result result = new failRequest_result();
+            if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, failRequest_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.failRequest(args.id,resultHandler);
+      }
+    }
+
+    public static class failRequestV2<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, failRequestV2_args, Void> {
+      public failRequestV2() {
+        super("failRequestV2");
+      }
+
+      public failRequestV2_args getEmptyArgsInstance() {
+        return new failRequestV2_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            failRequestV2_result result = new failRequestV2_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            failRequestV2_result result = new failRequestV2_result();
+            if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, failRequestV2_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.failRequestV2(args.id, args.e,resultHandler);
+      }
+    }
+
+  }
+
+  public static class result_args implements org.apache.thrift.TBase<result_args, result_args._Fields>, java.io.Serializable, Cloneable, Comparable<result_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("result_args");
+
+    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField RESULT_FIELD_DESC = new org.apache.thrift.protocol.TField("result", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new result_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new result_argsTupleSchemeFactory());
+    }
+
+    private String id; // required
+    private String result; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      ID((short)1, "id"),
+      RESULT((short)2, "result");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          case 2: // RESULT
+            return RESULT;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.RESULT, new org.apache.thrift.meta_data.FieldMetaData("result", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(result_args.class, metaDataMap);
+    }
+
+    public result_args() {
+    }
+
+    public result_args(
+      String id,
+      String result)
+    {
+      this();
+      this.id = id;
+      this.result = result;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public result_args(result_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+      if (other.is_set_result()) {
+        this.result = other.result;
+      }
+    }
+
+    public result_args deepCopy() {
+      return new result_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.id = null;
+      this.result = null;
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public String get_result() {
+      return this.result;
+    }
+
+    public void set_result(String result) {
+      this.result = result;
+    }
+
+    public void unset_result() {
+      this.result = null;
+    }
+
+    /** Returns true if field result is set (has been assigned a value) and false otherwise */
+    public boolean is_set_result() {
+      return this.result != null;
+    }
+
+    public void set_result_isSet(boolean value) {
+      if (!value) {
+        this.result = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      case RESULT:
+        if (value == null) {
+          unset_result();
+        } else {
+          set_result((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      case RESULT:
+        return get_result();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case ID:
+        return is_set_id();
+      case RESULT:
+        return is_set_result();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof result_args)
+        return this.equals((result_args)that);
+      return false;
+    }
+
+    public boolean equals(result_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      boolean this_present_result = true && this.is_set_result();
+      boolean that_present_result = true && that.is_set_result();
+      if (this_present_result || that_present_result) {
+        if (!(this_present_result && that_present_result))
+          return false;
+        if (!this.result.equals(that.result))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_id = true && (is_set_id());
+      list.add(present_id);
+      if (present_id)
+        list.add(id);
+
+      boolean present_result = true && (is_set_result());
+      list.add(present_result);
+      if (present_result)
+        list.add(result);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(result_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_result()).compareTo(other.is_set_result());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_result()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.result, other.result);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("result_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("result:");
+      if (this.result == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.result);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class result_argsStandardSchemeFactory implements SchemeFactory {
+      public result_argsStandardScheme getScheme() {
+        return new result_argsStandardScheme();
+      }
+    }
+
+    private static class result_argsStandardScheme extends StandardScheme<result_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, result_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.id = iprot.readString();
+                struct.set_id_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // RESULT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.result = iprot.readString();
+                struct.set_result_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, result_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.id != null) {
+          oprot.writeFieldBegin(ID_FIELD_DESC);
+          oprot.writeString(struct.id);
+          oprot.writeFieldEnd();
+        }
+        if (struct.result != null) {
+          oprot.writeFieldBegin(RESULT_FIELD_DESC);
+          oprot.writeString(struct.result);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class result_argsTupleSchemeFactory implements SchemeFactory {
+      public result_argsTupleScheme getScheme() {
+        return new result_argsTupleScheme();
+      }
+    }
+
+    private static class result_argsTupleScheme extends TupleScheme<result_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, result_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_id()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_result()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_id()) {
+          oprot.writeString(struct.id);
+        }
+        if (struct.is_set_result()) {
+          oprot.writeString(struct.result);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, result_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.id = iprot.readString();
+          struct.set_id_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.result = iprot.readString();
+          struct.set_result_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class result_result implements org.apache.thrift.TBase<result_result, result_result._Fields>, java.io.Serializable, Cloneable, Comparable<result_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("result_result");
+
+    private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new result_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new result_resultTupleSchemeFactory());
+    }
+
+    private AuthorizationException aze; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      AZE((short)1, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // AZE
+            return AZE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(result_result.class, metaDataMap);
+    }
+
+    public result_result() {
+    }
+
+    public result_result(
+      AuthorizationException aze)
+    {
+      this();
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public result_result(result_result other) {
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public result_result deepCopy() {
+      return new result_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.aze = null;
+    }
+
+    public AuthorizationException get_aze() {
+      return this.aze;
+    }
+
+    public void set_aze(AuthorizationException aze) {
+      this.aze = aze;
+    }
+
+    public void unset_aze() {
+      this.aze = null;
+    }
+
+    /** Returns true if field aze is set (has been assigned a value) and false otherwise */
+    public boolean is_set_aze() {
+      return this.aze != null;
+    }
+
+    public void set_aze_isSet(boolean value) {
+      if (!value) {
+        this.aze = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case AZE:
+        return get_aze();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case AZE:
+        return is_set_aze();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof result_result)
+        return this.equals((result_result)that);
+      return false;
+    }
+
+    public boolean equals(result_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_aze = true && this.is_set_aze();
+      boolean that_present_aze = true && that.is_set_aze();
+      if (this_present_aze || that_present_aze) {
+        if (!(this_present_aze && that_present_aze))
+          return false;
+        if (!this.aze.equals(that.aze))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_aze = true && (is_set_aze());
+      list.add(present_aze);
+      if (present_aze)
+        list.add(aze);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(result_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("result_result(");
+      boolean first = true;
+
+      sb.append("aze:");
+      if (this.aze == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.aze);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class result_resultStandardSchemeFactory implements SchemeFactory {
+      public result_resultStandardScheme getScheme() {
+        return new result_resultStandardScheme();
+      }
+    }
+
+    private static class result_resultStandardScheme extends StandardScheme<result_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, result_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // AZE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.aze = new AuthorizationException();
+                struct.aze.read(iprot);
+                struct.set_aze_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, result_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.aze != null) {
+          oprot.writeFieldBegin(AZE_FIELD_DESC);
+          struct.aze.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class result_resultTupleSchemeFactory implements SchemeFactory {
+      public result_resultTupleScheme getScheme() {
+        return new result_resultTupleScheme();
+      }
+    }
+
+    private static class result_resultTupleScheme extends TupleScheme<result_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, result_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_aze()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.is_set_aze()) {
+          struct.aze.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, result_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.aze = new AuthorizationException();
+          struct.aze.read(iprot);
+          struct.set_aze_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class fetchRequest_args implements org.apache.thrift.TBase<fetchRequest_args, fetchRequest_args._Fields>, java.io.Serializable, Cloneable, Comparable<fetchRequest_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fetchRequest_args");
+
+    private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new fetchRequest_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new fetchRequest_argsTupleSchemeFactory());
+    }
+
+    private String functionName; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      FUNCTION_NAME((short)1, "functionName");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // FUNCTION_NAME
+            return FUNCTION_NAME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_args.class, metaDataMap);
+    }
+
+    public fetchRequest_args() {
+    }
+
+    public fetchRequest_args(
+      String functionName)
+    {
+      this();
+      this.functionName = functionName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public fetchRequest_args(fetchRequest_args other) {
+      if (other.is_set_functionName()) {
+        this.functionName = other.functionName;
+      }
+    }
+
+    public fetchRequest_args deepCopy() {
+      return new fetchRequest_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.functionName = null;
+    }
+
+    public String get_functionName() {
+      return this.functionName;
+    }
+
+    public void set_functionName(String functionName) {
+      this.functionName = functionName;
+    }
+
+    public void unset_functionName() {
+      this.functionName = null;
+    }
+
+    /** Returns true if field functionName is set (has been assigned a value) and false otherwise */
+    public boolean is_set_functionName() {
+      return this.functionName != null;
+    }
+
+    public void set_functionName_isSet(boolean value) {
+      if (!value) {
+        this.functionName = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case FUNCTION_NAME:
+        if (value == null) {
+          unset_functionName();
+        } else {
+          set_functionName((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case FUNCTION_NAME:
+        return get_functionName();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case FUNCTION_NAME:
+        return is_set_functionName();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof fetchRequest_args)
+        return this.equals((fetchRequest_args)that);
+      return false;
+    }
+
+    public boolean equals(fetchRequest_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_functionName = true && this.is_set_functionName();
+      boolean that_present_functionName = true && that.is_set_functionName();
+      if (this_present_functionName || that_present_functionName) {
+        if (!(this_present_functionName && that_present_functionName))
+          return false;
+        if (!this.functionName.equals(that.functionName))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_functionName = true && (is_set_functionName());
+      list.add(present_functionName);
+      if (present_functionName)
+        list.add(functionName);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(fetchRequest_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(other.is_set_functionName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_functionName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, other.functionName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("fetchRequest_args(");
+      boolean first = true;
+
+      sb.append("functionName:");
+      if (this.functionName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.functionName);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class fetchRequest_argsStandardSchemeFactory implements SchemeFactory {
+      public fetchRequest_argsStandardScheme getScheme() {
+        return new fetchRequest_argsStandardScheme();
+      }
+    }
+
+    private static class fetchRequest_argsStandardScheme extends StandardScheme<fetchRequest_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, fetchRequest_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // FUNCTION_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.functionName = iprot.readString();
+                struct.set_functionName_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, fetchRequest_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.functionName != null) {
+          oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC);
+          oprot.writeString(struct.functionName);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class fetchRequest_argsTupleSchemeFactory implements SchemeFactory {
+      public fetchRequest_argsTupleScheme getScheme() {
+        return new fetchRequest_argsTupleScheme();
+      }
+    }
+
+    private static class fetchRequest_argsTupleScheme extends TupleScheme<fetchRequest_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, fetchRequest_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_functionName()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.is_set_functionName()) {
+          oprot.writeString(struct.functionName);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, fetchRequest_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.functionName = iprot.readString();
+          struct.set_functionName_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class fetchRequest_result implements org.apache.thrift.TBase<fetchRequest_result, fetchRequest_result._Fields>, java.io.Serializable, Cloneable, Comparable<fetchRequest_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fetchRequest_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new fetchRequest_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new fetchRequest_resultTupleSchemeFactory());
+    }
+
+    private DRPCRequest success; // required
+    private AuthorizationException aze; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // AZE
+            return AZE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DRPCRequest.class)));
+      tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_result.class, metaDataMap);
+    }
+
+    public fetchRequest_result() {
+    }
+
+    public fetchRequest_result(
+      DRPCRequest success,
+      AuthorizationException aze)
+    {
+      this();
+      this.success = success;
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public fetchRequest_result(fetchRequest_result other) {
+      if (other.is_set_success()) {
+        this.success = new DRPCRequest(other.success);
+      }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public fetchRequest_result deepCopy() {
+      return new fetchRequest_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.aze = null;
+    }
+
+    public DRPCRequest get_success() {
+      return this.success;
+    }
+
+    public void set_success(DRPCRequest success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public AuthorizationException get_aze() {
+      return this.aze;
+    }
+
+    public void set_aze(AuthorizationException aze) {
+      this.aze = aze;
+    }
+
+    public void unset_aze() {
+      this.aze = null;
+    }
+
+    /** Returns true if field aze is set (has been assigned a value) and false otherwise */
+    public boolean is_set_aze() {
+      return this.aze != null;
+    }
+
+    public void set_aze_isSet(boolean value) {
+      if (!value) {
+        this.aze = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((DRPCRequest)value);
+        }
+        break;
+
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case AZE:
+        return get_aze();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case AZE:
+        return is_set_aze();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof fetchRequest_result)
+        return this.equals((fetchRequest_result)that);
+      return false;
+    }
+
+    public boolean equals(fetchRequest_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_aze = true && this.is_set_aze();
+      boolean that_present_aze = true && that.is_set_aze();
+      if (this_present_aze || that_present_aze) {
+        if (!(this_present_aze && that_present_aze))
+          return false;
+        if (!this.aze.equals(that.aze))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (is_set_success());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_aze = true && (is_set_aze());
+      list.add(present_aze);
+      if (present_aze)
+        list.add(aze);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(fetchRequest_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("fetchRequest_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("aze:");
+      if (this.aze == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.aze);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class fetchRequest_resultStandardSchemeFactory implements SchemeFactory {
+      public fetchRequest_resultStandardScheme getScheme() {
+        return new fetchRequest_resultStandardScheme();
+      }
+    }
+
+    private static class fetchRequest_resultStandardScheme extends StandardScheme<fetchRequest_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, fetchRequest_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new DRPCRequest();
+                struct.success.read(iprot);
+                struct.set_success_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // AZE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.aze = new AuthorizationException();
+                struct.aze.read(iprot);
+                struct.set_aze_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, fetchRequest_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.aze != null) {
+          oprot.writeFieldBegin(AZE_FIELD_DESC);
+          struct.aze.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class fetchRequest_resultTupleSchemeFactory implements SchemeFactory {
+      public fetchRequest_resultTupleScheme getScheme() {
+        return new fetchRequest_resultTupleScheme();
+      }
+    }
+
+    private static class fetchRequest_resultTupleScheme extends TupleScheme<fetchRequest_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, fetchRequest_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_success()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_aze()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_success()) {
+          struct.success.write(oprot);
+        }
+        if (struct.is_set_aze()) {
+          struct.aze.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, fetchRequest_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = new DRPCRequest();
+          struct.success.read(iprot);
+          struct.set_success_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.aze = new AuthorizationException();
+          struct.aze.read(iprot);
+          struct.set_aze_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class failRequest_args implements org.apache.thrift.TBase<failRequest_args, failRequest_args._Fields>, java.io.Serializable, Cloneable, Comparable<failRequest_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("failRequest_args");
+
+    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new failRequest_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new failRequest_argsTupleSchemeFactory());
+    }
+
+    private String id; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      ID((short)1, "id");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_args.class, metaDataMap);
+    }
+
+    public failRequest_args() {
+    }
+
+    public failRequest_args(
+      String id)
+    {
+      this();
+      this.id = id;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public failRequest_args(failRequest_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+    }
+
+    public failRequest_args deepCopy() {
+      return new failRequest_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.id = null;
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and fa

<TRUNCATED>

[52/52] storm git commit: Merge branch 'STORM-2441' of https://github.com/HeartSaVioR/storm into STORM-2441

Posted by bo...@apache.org.
Merge branch 'STORM-2441' of https://github.com/HeartSaVioR/storm into STORM-2441

STORM-2441 Break down 'storm-core' to extract client (worker) artifacts


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

Branch: refs/heads/master
Commit: 4eb65075ad066da9f8bb25c47503a10546a30458
Parents: c652d3f 4de339a
Author: Robert Evans <ev...@yahoo-inc.com>
Authored: Thu Apr 6 08:33:21 2017 -0500
Committer: Robert Evans <ev...@yahoo-inc.com>
Committed: Thu Apr 6 08:33:21 2017 -0500

----------------------------------------------------------------------
 .travis.yml                                     |     4 +-
 DEVELOPER.md                                    |     2 +-
 bin/storm.py                                    |     5 +-
 ...Defining-a-non-jvm-language-dsl-for-storm.md |     2 +-
 docs/Lifecycle-of-a-topology.md                 |     2 +-
 docs/Maven.md                                   |     2 +-
 docs/Metrics.md                                 |    26 +-
 docs/Storm-Scheduler.md                         |     4 +-
 docs/Stream-API.md                              |     2 +-
 docs/Structure-of-the-codebase.md               |    80 +-
 docs/Trident-API-Overview.md                    |     2 +-
 docs/Trident-spouts.md                          |     8 +-
 docs/Trident-state.md                           |     8 +-
 docs/Windowing.md                               |     2 +-
 docs/storm-kinesis.md                           |     2 +-
 examples/storm-elasticsearch-examples/pom.xml   |     2 +-
 examples/storm-hbase-examples/pom.xml           |     2 +-
 examples/storm-hdfs-examples/pom.xml            |     2 +-
 .../storm/hdfs/spout/HdfsSpoutTopology.java     |     2 +-
 examples/storm-hive-examples/pom.xml            |     2 +-
 examples/storm-jdbc-examples/pom.xml            |     2 +-
 examples/storm-jms-examples/pom.xml             |     4 +-
 examples/storm-kafka-client-examples/pom.xml    |     2 +-
 examples/storm-kafka-examples/pom.xml           |     2 +-
 .../storm/kafka/trident/DrpcResultsPrinter.java |     4 +-
 examples/storm-mongodb-examples/pom.xml         |     2 +-
 .../storm/mongodb/topology/WordCounter.java     |     2 -
 examples/storm-mqtt-examples/pom.xml            |     2 +-
 examples/storm-opentsdb-examples/pom.xml        |     2 +-
 examples/storm-perf/pom.xml                     |     2 +-
 .../org/apache/storm/perf/KafkaHdfsTopo.java    |     3 +-
 .../storm/perf/KafkaSpoutNullBoltTopo.java      |     3 +-
 .../storm/perf/utils/BasicMetricsCollector.java |     1 -
 .../org/apache/storm/perf/utils/Helper.java     |     5 +-
 examples/storm-pmml-examples/pom.xml            |     2 +-
 examples/storm-redis-examples/pom.xml           |     2 +-
 .../storm/redis/topology/WordCounter.java       |     2 -
 examples/storm-solr-examples/pom.xml            |     2 +-
 examples/storm-starter/README.markdown          |     2 +-
 examples/storm-starter/pom.xml                  |    13 +-
 .../apache/storm/starter/clj/exclamation.clj    |     6 +-
 .../storm/starter/clj/rolling_top_words.clj     |     8 +-
 .../starter/BlobStoreAPIWordCountTopology.java  |     3 -
 .../storm/starter/FastWordCountTopology.java    |     3 +-
 .../storm/starter/InOrderDeliveryTest.java      |     4 +-
 .../storm/starter/MultipleLoggerTopology.java   |     1 -
 .../apache/storm/starter/StatefulTopology.java  |   143 +
 .../starter/StatefulWindowingTopology.java      |   111 +
 .../storm/starter/ThroughputVsLatency.java      |    10 +-
 .../storm/starter/streams/AggregateExample.java |     8 -
 .../src/jvm/storm/starter/StatefulTopology.java |   144 -
 .../starter/StatefulWindowingTopology.java      |   112 -
 external/storm-cassandra/pom.xml                |     9 +-
 .../storm/cassandra/bolt/BaseCassandraBolt.java |     2 -
 .../storm/cassandra/client/CassandraConf.java   |    15 +-
 external/storm-druid/pom.xml                    |     8 +-
 external/storm-elasticsearch/pom.xml            |     2 +-
 external/storm-eventhubs/pom.xml                |     9 +-
 .../eventhubs/samples/AtMostOnceEventCount.java |    54 -
 .../storm/eventhubs/samples/EventCount.java     |   157 -
 .../storm/eventhubs/samples/EventHubLoop.java   |    52 -
 .../samples/OpaqueTridentEventCount.java        |    53 -
 .../samples/TransactionalTridentEventCount.java |    81 -
 .../eventhubs/samples/bolt/GlobalCountBolt.java |    88 -
 .../samples/bolt/PartialCountBolt.java          |    68 -
 .../eventhubs/samples/AtMostOnceEventCount.java |    54 +
 .../storm/eventhubs/samples/EventCount.java     |   157 +
 .../storm/eventhubs/samples/EventHubLoop.java   |    52 +
 .../samples/OpaqueTridentEventCount.java        |    53 +
 .../samples/TransactionalTridentEventCount.java |    81 +
 .../eventhubs/samples/bolt/GlobalCountBolt.java |    88 +
 .../samples/bolt/PartialCountBolt.java          |    68 +
 external/storm-hbase/pom.xml                    |     2 +-
 external/storm-hdfs/pom.xml                     |     6 +-
 .../storm/hdfs/blobstore/HdfsBlobStoreImpl.java |     4 +-
 .../storm/hdfs/blobstore/BlobStoreTest.java     |     6 -
 external/storm-hive/pom.xml                     |     4 +-
 external/storm-jdbc/pom.xml                     |     2 +-
 external/storm-jms/pom.xml                      |     2 +-
 .../org/apache/storm/jms/spout/JmsSpout.java    |     2 +-
 .../storm/jms/trident/TridentJmsSpout.java      |     2 +-
 external/storm-kafka-client/pom.xml             |     9 +-
 external/storm-kafka/pom.xml                    |     2 +-
 .../storm/kafka/DynamicBrokersReader.java       |    12 +-
 .../src/jvm/org/apache/storm/kafka/ZkState.java |    10 +-
 .../org/apache/storm/kafka/KafkaUtilsTest.java  |     2 +-
 .../test/org/apache/storm/kafka/TestUtils.java  |     2 -
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  |     2 +-
 external/storm-kinesis/README.md                |     2 +-
 external/storm-kinesis/pom.xml                  |     2 +-
 external/storm-metrics/pom.xml                  |     2 +-
 external/storm-mongodb/pom.xml                  |     6 +-
 external/storm-mqtt/pom.xml                     |     8 +-
 external/storm-opentsdb/pom.xml                 |     2 +-
 external/storm-pmml/pom.xml                     |     2 +-
 external/storm-redis/pom.xml                    |     2 +-
 external/storm-solr/pom.xml                     |     2 +-
 flux/README.md                                  |    22 +
 .../apache/storm/flux/examples/WordCounter.java |     1 -
 flux/pom.xml                                    |     9 +-
 integration-test/pom.xml                        |     2 +-
 .../st/tests/window/SlidingWindowTest.java      |     2 +-
 .../apache/storm/st/wrapper/StormCluster.java   |     2 +-
 .../org/apache/storm/st/wrapper/TopoWrap.java   |     2 +-
 pom.xml                                         |     5 +-
 sql/storm-sql-core/pom.xml                      |     8 +-
 sql/storm-sql-external/storm-sql-hdfs/pom.xml   |     2 +-
 sql/storm-sql-external/storm-sql-kafka/pom.xml  |     2 +-
 .../storm-sql-mongodb/pom.xml                   |     2 +-
 sql/storm-sql-external/storm-sql-redis/pom.xml  |     2 +-
 sql/storm-sql-runtime/pom.xml                   |     2 +-
 .../storm/sql/runtime/utils/SerdeUtils.java     |     6 +-
 storm-client-misc/pom.xml                       |    50 +
 .../metric/HttpForwardingMetricsConsumer.java   |    85 +
 .../metric/HttpForwardingMetricsServer.java     |   118 +
 storm-client/pom.xml                            |   336 +
 storm-client/src/genthrift.sh                   |    34 +
 storm-client/src/java_license_header.txt        |    17 +
 .../src/jvm/org/apache/storm/Config.java        |  1697 +
 .../src/jvm/org/apache/storm/Constants.java     |    60 +
 .../org/apache/storm/ICredentialsListener.java  |    32 +
 .../src/jvm/org/apache/storm/ILocalCluster.java |   176 +
 .../src/jvm/org/apache/storm/ILocalDRPC.java    |    36 +
 .../jvm/org/apache/storm/ISubmitterHook.java    |    31 +
 .../src/jvm/org/apache/storm/LogWriter.java     |    83 +
 .../jvm/org/apache/storm/StormSubmitter.java    |   603 +
 .../src/jvm/org/apache/storm/StormTimer.java    |   242 +
 .../src/jvm/org/apache/storm/Thrift.java        |   350 +
 .../storm/annotation/InterfaceStability.java    |    54 +
 .../storm/blobstore/AtomicOutputStream.java     |    32 +
 .../org/apache/storm/blobstore/BlobStore.java   |   490 +
 .../storm/blobstore/BlobStoreAclHandler.java    |   409 +
 .../apache/storm/blobstore/BlobStoreFile.java   |    50 +
 .../apache/storm/blobstore/ClientBlobStore.java |   201 +
 .../storm/blobstore/InputStreamWithMeta.java    |    26 +
 .../org/apache/storm/blobstore/KeyFilter.java   |    22 +
 .../apache/storm/blobstore/NimbusBlobStore.java |   428 +
 .../src/jvm/org/apache/storm/bolt/JoinBolt.java |   595 +
 .../storm/callback/DefaultWatcherCallBack.java  |    35 +
 .../apache/storm/callback/WatcherCallBack.java  |    24 +
 .../storm/callback/ZKStateChangedCallback.java  |    25 +
 .../org/apache/storm/clojure/ClojureBolt.java   |   120 +
 .../org/apache/storm/clojure/ClojureSpout.java  |   153 +
 .../org/apache/storm/clojure/RichShellBolt.java |    51 +
 .../apache/storm/clojure/RichShellSpout.java    |    51 +
 .../storm/cluster/ClusterStateContext.java      |    41 +
 .../storm/cluster/ClusterStateListener.java     |    22 +
 .../org/apache/storm/cluster/ClusterUtils.java  |   244 +
 .../apache/storm/cluster/ConnectionState.java   |    24 +
 .../org/apache/storm/cluster/DaemonType.java    |    27 +
 .../org/apache/storm/cluster/ExecutorBeat.java  |    44 +
 .../org/apache/storm/cluster/IStateStorage.java |   219 +
 .../storm/cluster/IStormClusterState.java       |   198 +
 .../storm/cluster/PaceMakerStateStorage.java    |   265 +
 .../cluster/PaceMakerStateStorageFactory.java   |    37 +
 .../storm/cluster/StateStorageFactory.java      |    28 +
 .../storm/cluster/StormClusterStateImpl.java    |   735 +
 .../org/apache/storm/cluster/VersionedData.java |    36 +
 .../apache/storm/cluster/ZKStateStorage.java    |   252 +
 .../storm/cluster/ZKStateStorageFactory.java    |    36 +
 .../storm/container/cgroup/CgroupCenter.java    |   216 +
 .../storm/container/cgroup/CgroupCommon.java    |   270 +
 .../container/cgroup/CgroupCommonOperation.java |    81 +
 .../container/cgroup/CgroupCoreFactory.java     |    72 +
 .../storm/container/cgroup/CgroupOperation.java |    79 +
 .../storm/container/cgroup/CgroupUtils.java     |   118 +
 .../apache/storm/container/cgroup/Device.java   |    75 +
 .../storm/container/cgroup/Hierarchy.java       |   130 +
 .../storm/container/cgroup/SubSystem.java       |    81 +
 .../storm/container/cgroup/SubSystemType.java   |    36 +
 .../storm/container/cgroup/SystemOperation.java |    76 +
 .../storm/container/cgroup/core/BlkioCore.java  |   213 +
 .../storm/container/cgroup/core/CgroupCore.java |    26 +
 .../storm/container/cgroup/core/CpuCore.java    |   135 +
 .../container/cgroup/core/CpuacctCore.java      |    71 +
 .../storm/container/cgroup/core/CpusetCore.java |   209 +
 .../container/cgroup/core/DevicesCore.java      |   189 +
 .../container/cgroup/core/FreezerCore.java      |    66 +
 .../storm/container/cgroup/core/MemoryCore.java |   188 +
 .../storm/container/cgroup/core/NetClsCore.java |    69 +
 .../container/cgroup/core/NetPrioCore.java      |    65 +
 .../storm/coordination/BatchBoltExecutor.java   |   109 +
 .../coordination/BatchOutputCollector.java      |    46 +
 .../coordination/BatchOutputCollectorImpl.java  |    53 +
 .../coordination/BatchSubtopologyBuilder.java   |   447 +
 .../storm/coordination/CoordinatedBolt.java     |   385 +
 .../apache/storm/coordination/IBatchBolt.java   |    30 +
 .../src/jvm/org/apache/storm/daemon/Acker.java  |   139 +
 .../org/apache/storm/daemon/DaemonCommon.java   |    22 +
 .../org/apache/storm/daemon/GrouperFactory.java |   243 +
 .../org/apache/storm/daemon/Shutdownable.java   |    22 +
 .../org/apache/storm/daemon/StormCommon.java    |   594 +
 .../src/jvm/org/apache/storm/daemon/Task.java   |   247 +
 .../daemon/metrics/BuiltinBoltMetrics.java      |    78 +
 .../storm/daemon/metrics/BuiltinMetrics.java    |    33 +
 .../daemon/metrics/BuiltinMetricsUtil.java      |    81 +
 .../daemon/metrics/BuiltinSpoutMetrics.java     |    64 +
 .../daemon/metrics/SpoutThrottlingMetrics.java  |    57 +
 .../storm/daemon/supervisor/AdvancedFSOps.java  |   367 +
 .../supervisor/ClientSupervisorUtils.java       |   167 +
 .../daemon/supervisor/ExitCodeCallback.java     |    30 +
 .../storm/daemon/supervisor/IAdvancedFSOps.java |   173 +
 .../storm/daemon/worker/LogConfigManager.java   |   156 +
 .../org/apache/storm/daemon/worker/Worker.java  |   459 +
 .../apache/storm/daemon/worker/WorkerState.java |   690 +
 .../dependency/DependencyBlobStoreUtils.java    |    43 +
 .../dependency/DependencyPropertiesParser.java  |    61 +
 .../storm/dependency/DependencyUploader.java    |   167 +
 .../dependency/FileNotAvailableException.java   |    33 +
 .../storm/drpc/DRPCInvocationsClient.java       |   130 +
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |   287 +
 .../jvm/org/apache/storm/drpc/JoinResult.java   |    75 +
 .../org/apache/storm/drpc/KeyedFairBolt.java    |    93 +
 .../storm/drpc/LinearDRPCInputDeclarer.java     |    52 +
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |   393 +
 .../org/apache/storm/drpc/PrepareRequest.java   |    60 +
 .../org/apache/storm/drpc/ReturnResults.java    |   138 +
 .../jvm/org/apache/storm/executor/Executor.java |   583 +
 .../apache/storm/executor/ExecutorShutdown.java |   114 +
 .../apache/storm/executor/ExecutorTransfer.java |    88 +
 .../apache/storm/executor/IRunningExecutor.java |    31 +
 .../apache/storm/executor/LocalExecutor.java    |    56 +
 .../org/apache/storm/executor/TupleInfo.java    |    90 +
 .../storm/executor/bolt/BoltExecutor.java       |   139 +
 .../executor/bolt/BoltOutputCollectorImpl.java  |   171 +
 .../storm/executor/error/IReportError.java      |    22 +
 .../storm/executor/error/ReportError.java       |    77 +
 .../storm/executor/error/ReportErrorAndDie.java |    49 +
 .../storm/executor/spout/SpoutExecutor.java     |   259 +
 .../spout/SpoutOutputCollectorImpl.java         |   147 +
 .../apache/storm/generated/AccessControl.java   |   627 +
 .../storm/generated/AccessControlType.java      |    62 +
 .../storm/generated/AlreadyAliveException.java  |   406 +
 .../org/apache/storm/generated/Assignment.java  |  1159 +
 .../storm/generated/AuthorizationException.java |   406 +
 .../storm/generated/BeginDownloadResult.java    |   608 +
 .../jvm/org/apache/storm/generated/Bolt.java    |   514 +
 .../storm/generated/BoltAggregateStats.java     |   704 +
 .../org/apache/storm/generated/BoltStats.java   |  1390 +
 .../apache/storm/generated/ClusterSummary.java  |   879 +
 .../storm/generated/ClusterWorkerHeartbeat.java |   768 +
 .../storm/generated/CommonAggregateStats.java   |  1060 +
 .../generated/ComponentAggregateStats.java      |   752 +
 .../apache/storm/generated/ComponentCommon.java |   852 +
 .../apache/storm/generated/ComponentObject.java |   462 +
 .../storm/generated/ComponentPageInfo.java      |  2352 +
 .../apache/storm/generated/ComponentType.java   |    62 +
 .../org/apache/storm/generated/Credentials.java |   458 +
 .../storm/generated/DRPCExceptionType.java      |    68 +
 .../storm/generated/DRPCExecutionException.java |   528 +
 .../org/apache/storm/generated/DRPCRequest.java |   507 +
 .../apache/storm/generated/DebugOptions.java    |   506 +
 .../apache/storm/generated/DistributedRPC.java  |  1328 +
 .../generated/DistributedRPCInvocations.java    |  3904 ++
 .../org/apache/storm/generated/ErrorInfo.java   |   714 +
 .../storm/generated/ExecutorAggregateStats.java |   526 +
 .../apache/storm/generated/ExecutorInfo.java    |   499 +
 .../storm/generated/ExecutorSpecificStats.java  |   387 +
 .../apache/storm/generated/ExecutorStats.java   |   915 +
 .../apache/storm/generated/ExecutorSummary.java |   922 +
 .../apache/storm/generated/GetInfoOptions.java  |   422 +
 .../apache/storm/generated/GlobalStreamId.java  |   507 +
 .../org/apache/storm/generated/Grouping.java    |   800 +
 .../generated/HBAuthorizationException.java     |   406 +
 .../storm/generated/HBExecutionException.java   |   406 +
 .../org/apache/storm/generated/HBMessage.java   |   636 +
 .../apache/storm/generated/HBMessageData.java   |   640 +
 .../jvm/org/apache/storm/generated/HBNodes.java |   461 +
 .../jvm/org/apache/storm/generated/HBPulse.java |   522 +
 .../org/apache/storm/generated/HBRecords.java   |   466 +
 .../storm/generated/HBServerMessageType.java    |   113 +
 .../generated/InvalidTopologyException.java     |   406 +
 .../org/apache/storm/generated/JavaObject.java  |   561 +
 .../apache/storm/generated/JavaObjectArg.java   |   631 +
 .../generated/KeyAlreadyExistsException.java    |   406 +
 .../storm/generated/KeyNotFoundException.java   |   406 +
 .../org/apache/storm/generated/KillOptions.java |   407 +
 .../storm/generated/LSApprovedWorkers.java      |   458 +
 .../generated/LSSupervisorAssignments.java      |   471 +
 .../apache/storm/generated/LSSupervisorId.java  |   406 +
 .../apache/storm/generated/LSTopoHistory.java   |   805 +
 .../storm/generated/LSTopoHistoryList.java      |   460 +
 .../storm/generated/LSWorkerHeartbeat.java      |   755 +
 .../apache/storm/generated/ListBlobsResult.java |   556 +
 .../apache/storm/generated/LocalAssignment.java |   676 +
 .../apache/storm/generated/LocalStateData.java  |   471 +
 .../org/apache/storm/generated/LogConfig.java   |   475 +
 .../org/apache/storm/generated/LogLevel.java    |   836 +
 .../apache/storm/generated/LogLevelAction.java  |    65 +
 .../jvm/org/apache/storm/generated/Nimbus.java  | 47241 +++++++++++++++++
 .../apache/storm/generated/NimbusSummary.java   |   796 +
 .../org/apache/storm/generated/NodeInfo.java    |   556 +
 .../storm/generated/NotAliveException.java      |   406 +
 .../org/apache/storm/generated/NullStruct.java  |   300 +
 .../apache/storm/generated/NumErrorsChoice.java |    65 +
 .../apache/storm/generated/ProfileAction.java   |    74 +
 .../apache/storm/generated/ProfileRequest.java  |   631 +
 .../storm/generated/ReadableBlobMeta.java       |   510 +
 .../storm/generated/RebalanceOptions.java       |   664 +
 .../storm/generated/SettableBlobMeta.java       |   567 +
 .../apache/storm/generated/ShellComponent.java  |   516 +
 .../storm/generated/SpecificAggregateStats.java |   387 +
 .../storm/generated/SpoutAggregateStats.java    |   407 +
 .../org/apache/storm/generated/SpoutSpec.java   |   514 +
 .../org/apache/storm/generated/SpoutStats.java  |   917 +
 .../apache/storm/generated/StateSpoutSpec.java  |   514 +
 .../org/apache/storm/generated/StormBase.java   |  1382 +
 .../apache/storm/generated/StormTopology.java   |  1272 +
 .../org/apache/storm/generated/StreamInfo.java  |   554 +
 .../apache/storm/generated/SubmitOptions.java   |   533 +
 .../apache/storm/generated/SupervisorInfo.java  |  1446 +
 .../storm/generated/SupervisorPageInfo.java     |   624 +
 .../storm/generated/SupervisorSummary.java      |  1265 +
 .../storm/generated/ThriftSerializedObject.java |   516 +
 .../storm/generated/TopologyActionOptions.java  |   387 +
 .../storm/generated/TopologyHistoryInfo.java    |   461 +
 .../apache/storm/generated/TopologyInfo.java    |  2144 +
 .../storm/generated/TopologyInitialStatus.java  |    62 +
 .../storm/generated/TopologyPageInfo.java       |  2757 +
 .../apache/storm/generated/TopologyStats.java   |  1094 +
 .../apache/storm/generated/TopologyStatus.java  |    68 +
 .../apache/storm/generated/TopologySummary.java |  1901 +
 .../apache/storm/generated/WorkerResources.java |   605 +
 .../apache/storm/generated/WorkerSummary.java   |  1880 +
 .../storm/grouping/CustomStreamGrouping.java    |    43 +
 .../src/jvm/org/apache/storm/grouping/Load.java |    77 +
 .../grouping/LoadAwareCustomStreamGrouping.java |    24 +
 .../grouping/LoadAwareShuffleGrouping.java      |    76 +
 .../org/apache/storm/grouping/LoadMapping.java  |    64 +
 .../storm/grouping/PartialKeyGrouping.java      |   106 +
 .../apache/storm/grouping/ShuffleGrouping.java  |    63 +
 .../org/apache/storm/hooks/BaseTaskHook.java    |    61 +
 .../org/apache/storm/hooks/BaseWorkerHook.java  |    51 +
 .../jvm/org/apache/storm/hooks/ITaskHook.java   |    38 +
 .../jvm/org/apache/storm/hooks/IWorkerHook.java |    44 +
 .../storm/hooks/SubmitterHookException.java     |    41 +
 .../apache/storm/hooks/info/BoltAckInfo.java    |    40 +
 .../storm/hooks/info/BoltExecuteInfo.java       |    40 +
 .../apache/storm/hooks/info/BoltFailInfo.java   |    40 +
 .../org/apache/storm/hooks/info/EmitInfo.java   |    44 +
 .../apache/storm/hooks/info/SpoutAckInfo.java   |    39 +
 .../apache/storm/hooks/info/SpoutFailInfo.java  |    39 +
 .../storm/messaging/ConnectionWithStatus.java   |    49 +
 .../DeserializingConnectionCallback.java        |    60 +
 .../org/apache/storm/messaging/IConnection.java |    63 +
 .../storm/messaging/IConnectionCallback.java    |    31 +
 .../org/apache/storm/messaging/IContext.java    |    59 +
 .../org/apache/storm/messaging/TaskMessage.java |    53 +
 .../storm/messaging/TransportFactory.java       |    57 +
 .../apache/storm/messaging/local/Context.java   |   218 +
 .../apache/storm/messaging/netty/Client.java    |   612 +
 .../apache/storm/messaging/netty/Context.java   |   113 +
 .../storm/messaging/netty/ControlMessage.java   |    75 +
 .../messaging/netty/INettySerializable.java     |    26 +
 .../storm/messaging/netty/ISaslClient.java      |    28 +
 .../storm/messaging/netty/ISaslServer.java      |    26 +
 .../apache/storm/messaging/netty/IServer.java   |    26 +
 .../netty/KerberosSaslClientHandler.java        |   154 +
 .../netty/KerberosSaslNettyClient.java          |   202 +
 .../netty/KerberosSaslNettyClientState.java     |    31 +
 .../netty/KerberosSaslNettyServer.java          |   209 +
 .../netty/KerberosSaslNettyServerState.java     |    30 +
 .../netty/KerberosSaslServerHandler.java        |   133 +
 .../org/apache/storm/messaging/netty/Login.java |   411 +
 .../storm/messaging/netty/MessageBatch.java     |   118 +
 .../storm/messaging/netty/MessageBuffer.java    |    58 +
 .../storm/messaging/netty/MessageDecoder.java   |   144 +
 .../storm/messaging/netty/MessageEncoder.java   |    43 +
 .../netty/NettyRenameThreadFactory.java         |    56 +
 .../netty/NettyUncaughtExceptionHandler.java    |    35 +
 .../storm/messaging/netty/SaslMessageToken.java |   114 +
 .../storm/messaging/netty/SaslNettyClient.java  |   154 +
 .../messaging/netty/SaslNettyClientState.java   |    31 +
 .../storm/messaging/netty/SaslNettyServer.java  |   157 +
 .../messaging/netty/SaslNettyServerState.java   |    30 +
 .../messaging/netty/SaslStormClientHandler.java |   158 +
 .../netty/SaslStormServerAuthorizeHandler.java  |    83 +
 .../messaging/netty/SaslStormServerHandler.java |   153 +
 .../apache/storm/messaging/netty/SaslUtils.java |    68 +
 .../apache/storm/messaging/netty/Server.java    |   274 +
 .../messaging/netty/StormClientHandler.java     |    90 +
 .../netty/StormClientPipelineFactory.java       |    56 +
 .../messaging/netty/StormServerHandler.java     |    74 +
 .../netty/StormServerPipelineFactory.java       |    57 +
 .../apache/storm/metric/EventLoggerBolt.java    |    67 +
 .../storm/metric/FileBasedEventLogger.java      |   122 +
 .../org/apache/storm/metric/IEventLogger.java   |    70 +
 .../storm/metric/LoggingMetricsConsumer.java    |    76 +
 .../storm/metric/MetricsConsumerBolt.java       |   146 +
 .../jvm/org/apache/storm/metric/SystemBolt.java |   161 +
 .../storm/metric/api/AssignableMetric.java      |    34 +
 .../apache/storm/metric/api/CombinedMetric.java |    38 +
 .../apache/storm/metric/api/CountMetric.java    |    39 +
 .../org/apache/storm/metric/api/ICombiner.java  |    23 +
 .../org/apache/storm/metric/api/IMetric.java    |    30 +
 .../storm/metric/api/IMetricsConsumer.java      |    80 +
 .../org/apache/storm/metric/api/IReducer.java   |    24 +
 .../storm/metric/api/IStatefulObject.java       |    22 +
 .../apache/storm/metric/api/MeanReducer.java    |    54 +
 .../storm/metric/api/MultiCountMetric.java      |    45 +
 .../storm/metric/api/MultiReducedMetric.java    |    50 +
 .../apache/storm/metric/api/ReducedMetric.java  |    38 +
 .../apache/storm/metric/api/StateMetric.java    |    31 +
 .../metric/api/rpc/AssignableShellMetric.java   |    30 +
 .../metric/api/rpc/CombinedShellMetric.java     |    31 +
 .../storm/metric/api/rpc/CountShellMetric.java  |    37 +
 .../storm/metric/api/rpc/IShellMetric.java      |    31 +
 .../metric/api/rpc/ReducedShellMetric.java      |    32 +
 .../apache/storm/metric/cgroup/CGroupCpu.java   |    70 +
 .../storm/metric/cgroup/CGroupCpuGuarantee.java |    52 +
 .../storm/metric/cgroup/CGroupMemoryLimit.java  |    39 +
 .../storm/metric/cgroup/CGroupMemoryUsage.java  |    39 +
 .../storm/metric/cgroup/CGroupMetricsBase.java  |   114 +
 .../storm/metric/filter/FilterByMetricName.java |   110 +
 .../storm/metric/filter/MetricsFilter.java      |    26 +
 .../metric/internal/CountStatAndMetric.java     |   211 +
 .../metric/internal/LatencyStatAndMetric.java   |   262 +
 .../storm/metric/internal/MetricStatTimer.java  |    27 +
 .../internal/MultiCountStatAndMetric.java       |   112 +
 .../internal/MultiLatencyStatAndMetric.java     |   109 +
 .../storm/metric/internal/RateTracker.java      |   139 +
 .../storm/metric/util/DataPointExpander.java    |    79 +
 .../jvm/org/apache/storm/multilang/BoltMsg.java |    79 +
 .../org/apache/storm/multilang/ISerializer.java |    82 +
 .../apache/storm/multilang/JsonSerializer.java  |   203 +
 .../storm/multilang/NoOutputException.java      |    40 +
 .../org/apache/storm/multilang/ShellMsg.java    |   184 +
 .../org/apache/storm/multilang/SpoutMsg.java    |    50 +
 .../AbstractDNSToSwitchMapping.java             |    96 +
 .../networktopography/DNSToSwitchMapping.java   |    49 +
 .../DefaultRackDNSToSwitchMapping.java          |    52 +
 .../jvm/org/apache/storm/nimbus/NimbusInfo.java |   120 +
 .../apache/storm/pacemaker/PacemakerClient.java |   274 +
 .../storm/pacemaker/PacemakerClientHandler.java |    80 +
 .../storm/pacemaker/PacemakerClientPool.java    |   113 +
 .../pacemaker/PacemakerConnectionException.java |    24 +
 .../storm/pacemaker/codec/ThriftDecoder.java    |    69 +
 .../storm/pacemaker/codec/ThriftEncoder.java    |   109 +
 .../pacemaker/codec/ThriftNettyClientCodec.java |    97 +
 .../jvm/org/apache/storm/scheduler/Cluster.java |   857 +
 .../apache/storm/scheduler/ExecutorDetails.java |    54 +
 .../jvm/org/apache/storm/scheduler/INimbus.java |    49 +
 .../org/apache/storm/scheduler/IScheduler.java  |    40 +
 .../org/apache/storm/scheduler/ISupervisor.java |    45 +
 .../storm/scheduler/SchedulerAssignment.java    |    61 +
 .../scheduler/SchedulerAssignmentImpl.java      |   135 +
 .../storm/scheduler/SupervisorDetails.java      |   138 +
 .../org/apache/storm/scheduler/Topologies.java  |    87 +
 .../apache/storm/scheduler/TopologyDetails.java |   517 +
 .../org/apache/storm/scheduler/WorkerSlot.java  |    81 +
 .../storm/scheduler/resource/Component.java     |    54 +
 .../storm/scheduler/resource/RAS_Node.java      |   529 +
 .../storm/scheduler/resource/RAS_Nodes.java     |   138 +
 .../storm/scheduler/resource/ResourceUtils.java |   207 +
 .../scheduler/resource/SchedulingResult.java    |   116 +
 .../scheduler/resource/SchedulingState.java     |    56 +
 .../scheduler/resource/SchedulingStatus.java    |    40 +
 .../apache/storm/scheduler/resource/User.java   |   350 +
 .../DefaultResourceAwareStrategy.java           |   757 +
 .../strategies/scheduling/IStrategy.java        |    47 +
 .../storm/security/INimbusCredentialPlugin.java |    47 +
 .../auth/AbstractSaslClientCallbackHandler.java |    76 +
 .../auth/AbstractSaslServerCallbackHandler.java |    94 +
 .../apache/storm/security/auth/AuthUtils.java   |   419 +
 .../org/apache/storm/security/auth/AutoSSL.java |   161 +
 .../auth/DefaultHttpCredentialsPlugin.java      |    96 +
 .../security/auth/DefaultPrincipalToLocal.java  |    42 +
 .../apache/storm/security/auth/IAuthorizer.java |    53 +
 .../storm/security/auth/IAutoCredentials.java   |    55 +
 .../security/auth/ICredentialsRenewer.java      |    40 +
 .../auth/IGroupMappingServiceProvider.java      |    42 +
 .../security/auth/IHttpCredentialsPlugin.java   |    48 +
 .../storm/security/auth/IPrincipalToLocal.java  |    41 +
 .../storm/security/auth/ITransportPlugin.java   |    57 +
 .../security/auth/KerberosPrincipalToLocal.java |    45 +
 .../storm/security/auth/NimbusPrincipal.java    |    29 +
 .../apache/storm/security/auth/ReqContext.java  |   152 +
 .../security/auth/SaslTransportPlugin.java      |   180 +
 .../security/auth/ShellBasedGroupsMapping.java  |    99 +
 .../security/auth/SimpleTransportPlugin.java    |   163 +
 .../security/auth/SingleUserPrincipal.java      |    53 +
 .../storm/security/auth/TBackoffConnect.java    |    82 +
 .../storm/security/auth/ThriftClient.java       |   124 +
 .../security/auth/ThriftConnectionType.java     |    86 +
 .../storm/security/auth/ThriftServer.java       |    78 +
 .../auth/authorizer/DRPCAuthorizerBase.java     |    63 +
 .../authorizer/DRPCSimpleACLAuthorizer.java     |   177 +
 .../auth/authorizer/DenyAuthorizer.java         |    47 +
 .../authorizer/ImpersonationAuthorizer.java     |   172 +
 .../auth/authorizer/NoopAuthorizer.java         |    47 +
 .../auth/authorizer/SimpleACLAuthorizer.java    |   173 +
 .../authorizer/SimpleWhitelistAuthorizer.java   |    60 +
 .../auth/digest/ClientCallbackHandler.java      |    58 +
 .../auth/digest/DigestSaslTransportPlugin.java  |    68 +
 .../auth/digest/ServerCallbackHandler.java      |    85 +
 .../storm/security/auth/kerberos/AutoTGT.java   |   269 +
 .../auth/kerberos/AutoTGTKrb5LoginModule.java   |   112 +
 .../kerberos/AutoTGTKrb5LoginModuleTest.java    |    44 +
 .../auth/kerberos/ClientCallbackHandler.java    |   104 +
 .../kerberos/KerberosSaslTransportPlugin.java   |   262 +
 .../security/auth/kerberos/NoOpTTrasport.java   |    54 +
 .../auth/kerberos/ServerCallbackHandler.java    |    96 +
 .../auth/kerberos/jaas_kerberos_cluster.conf    |    49 +
 .../auth/kerberos/jaas_kerberos_launcher.conf   |    31 +
 .../auth/plain/PlainClientCallbackHandler.java  |    31 +
 .../auth/plain/PlainSaslTransportPlugin.java    |    71 +
 .../auth/plain/PlainServerCallbackHandler.java  |    55 +
 .../security/auth/plain/SaslPlainServer.java    |   158 +
 .../serialization/BlowfishTupleSerializer.java  |    92 +
 .../storm/serialization/DefaultKryoFactory.java |    64 +
 .../DefaultSerializationDelegate.java           |    58 +
 .../GzipBridgeSerializationDelegate.java        |    65 +
 .../GzipBridgeThriftSerializationDelegate.java  |    64 +
 .../GzipSerializationDelegate.java              |    64 +
 .../GzipThriftSerializationDelegate.java        |    57 +
 .../storm/serialization/IKryoDecorator.java     |    23 +
 .../storm/serialization/IKryoFactory.java       |    40 +
 .../storm/serialization/ITupleDeserializer.java |    24 +
 .../storm/serialization/ITupleSerializer.java   |    26 +
 .../serialization/KryoTupleDeserializer.java    |    56 +
 .../serialization/KryoTupleSerializer.java      |    60 +
 .../serialization/KryoValuesDeserializer.java   |    50 +
 .../serialization/KryoValuesSerializer.java     |    58 +
 .../serialization/SerializableSerializer.java   |    61 +
 .../serialization/SerializationDelegate.java    |    35 +
 .../serialization/SerializationFactory.java     |   241 +
 .../ThriftSerializationDelegate.java            |    52 +
 .../types/ArrayListSerializer.java              |    32 +
 .../serialization/types/HashMapSerializer.java  |    32 +
 .../serialization/types/HashSetSerializer.java  |    32 +
 .../types/ListDelegateSerializer.java           |    32 +
 .../org/apache/storm/spout/CheckPointState.java |   172 +
 .../org/apache/storm/spout/CheckpointSpout.java |   232 +
 .../storm/spout/IMultiSchemableSpout.java       |    23 +
 .../org/apache/storm/spout/ISchemableSpout.java |    24 +
 .../src/jvm/org/apache/storm/spout/ISpout.java  |   105 +
 .../storm/spout/ISpoutOutputCollector.java      |    32 +
 .../apache/storm/spout/ISpoutWaitStrategy.java  |    34 +
 .../jvm/org/apache/storm/spout/MultiScheme.java |    29 +
 .../storm/spout/NothingEmptyEmitStrategy.java   |    31 +
 .../org/apache/storm/spout/RawMultiScheme.java  |    38 +
 .../jvm/org/apache/storm/spout/RawScheme.java   |    36 +
 .../src/jvm/org/apache/storm/spout/Scheme.java  |    29 +
 .../apache/storm/spout/SchemeAsMultiScheme.java |    42 +
 .../jvm/org/apache/storm/spout/ShellSpout.java  |   327 +
 .../storm/spout/SleepSpoutWaitStrategy.java     |    41 +
 .../storm/spout/SpoutOutputCollector.java       |   144 +
 .../storm/state/DefaultStateSerializer.java     |    77 +
 .../jvm/org/apache/storm/state/IStateSpout.java |    29 +
 .../storm/state/IStateSpoutOutputCollector.java |    22 +
 .../apache/storm/state/ISubscribedState.java    |    25 +
 .../state/ISynchronizeOutputCollector.java      |    24 +
 .../storm/state/InMemoryKeyValueState.java      |   126 +
 .../state/InMemoryKeyValueStateProvider.java    |    43 +
 .../org/apache/storm/state/KeyValueState.java   |    58 +
 .../jvm/org/apache/storm/state/Serializer.java  |    30 +
 .../src/jvm/org/apache/storm/state/State.java   |    55 +
 .../org/apache/storm/state/StateFactory.java    |    69 +
 .../org/apache/storm/state/StateProvider.java   |    38 +
 .../storm/state/StateSpoutOutputCollector.java  |    28 +
 .../storm/state/SynchronizeOutputCollector.java |    30 +
 .../apache/storm/stats/BoltExecutorStats.java   |   106 +
 .../jvm/org/apache/storm/stats/CommonStats.java |   114 +
 .../apache/storm/stats/SpoutExecutorStats.java  |    76 +
 .../jvm/org/apache/storm/stats/StatsUtil.java   |  2610 +
 .../src/jvm/org/apache/storm/streams/Edge.java  |    41 +
 .../org/apache/storm/streams/GroupingInfo.java  |   100 +
 .../src/jvm/org/apache/storm/streams/Node.java  |   170 +
 .../src/jvm/org/apache/storm/streams/Pair.java  |   103 +
 .../org/apache/storm/streams/PairStream.java    |   525 +
 .../org/apache/storm/streams/PartitionNode.java |    37 +
 .../org/apache/storm/streams/ProcessorBolt.java |    70 +
 .../storm/streams/ProcessorBoltDelegate.java    |   348 +
 .../org/apache/storm/streams/ProcessorNode.java |    83 +
 .../apache/storm/streams/RefCountedTuple.java   |    64 +
 .../jvm/org/apache/storm/streams/SinkNode.java  |    44 +
 .../jvm/org/apache/storm/streams/SpoutNode.java |    47 +
 .../storm/streams/StatefulProcessorBolt.java    |   116 +
 .../jvm/org/apache/storm/streams/Stream.java    |   515 +
 .../org/apache/storm/streams/StreamBolt.java    |    38 +
 .../org/apache/storm/streams/StreamBuilder.java |   602 +
 .../org/apache/storm/streams/StreamState.java   |    43 +
 .../org/apache/storm/streams/StreamUtil.java    |    69 +
 .../storm/streams/StreamsEdgeFactory.java       |    29 +
 .../org/apache/storm/streams/UniqueIdGen.java   |    56 +
 .../org/apache/storm/streams/WindowNode.java    |    39 +
 .../storm/streams/WindowedProcessorBolt.java    |   137 +
 .../storm/streams/operations/BiFunction.java    |    37 +
 .../streams/operations/CombinerAggregator.java  |    97 +
 .../storm/streams/operations/Consumer.java      |    32 +
 .../streams/operations/FlatMapFunction.java     |    27 +
 .../storm/streams/operations/Function.java      |    34 +
 .../streams/operations/IdentityFunction.java    |    31 +
 .../storm/streams/operations/Operation.java     |    26 +
 .../streams/operations/PairFlatMapFunction.java |    30 +
 .../storm/streams/operations/PairFunction.java  |    30 +
 .../streams/operations/PairValueJoiner.java     |    40 +
 .../storm/streams/operations/Predicate.java     |    33 +
 .../storm/streams/operations/PrintConsumer.java |    30 +
 .../storm/streams/operations/Reducer.java       |    35 +
 .../storm/streams/operations/StateUpdater.java  |    67 +
 .../storm/streams/operations/ValueJoiner.java   |    36 +
 .../streams/operations/aggregators/Count.java   |    47 +
 .../streams/operations/aggregators/LongSum.java |    45 +
 .../operations/mappers/PairValueMapper.java     |    51 +
 .../operations/mappers/TupleValueMapper.java    |    30 +
 .../operations/mappers/TupleValueMappers.java   |   174 +
 .../streams/operations/mappers/ValueMapper.java |    45 +
 .../operations/mappers/ValuesMapper.java        |    48 +
 .../processors/AggregateByKeyProcessor.java     |    78 +
 .../streams/processors/AggregateProcessor.java  |    69 +
 .../storm/streams/processors/BaseProcessor.java |   109 +
 .../streams/processors/BatchProcessor.java      |    25 +
 .../streams/processors/BranchProcessor.java     |    41 +
 .../processors/ChainedProcessorContext.java     |    66 +
 .../processors/EmittingProcessorContext.java    |   155 +
 .../streams/processors/FilterProcessor.java     |    35 +
 .../streams/processors/FlatMapProcessor.java    |    35 +
 .../processors/FlatMapValuesProcessor.java      |    36 +
 .../streams/processors/ForEachProcessor.java    |    33 +
 .../processors/ForwardingProcessorContext.java  |   102 +
 .../storm/streams/processors/JoinProcessor.java |   152 +
 .../storm/streams/processors/MapProcessor.java  |    33 +
 .../streams/processors/MapValuesProcessor.java  |    34 +
 .../MergeAggregateByKeyProcessor.java           |    54 +
 .../processors/MergeAggregateProcessor.java     |    47 +
 .../storm/streams/processors/PeekProcessor.java |    34 +
 .../storm/streams/processors/Processor.java     |    54 +
 .../streams/processors/ProcessorContext.java    |    62 +
 .../processors/ReduceByKeyProcessor.java        |    52 +
 .../streams/processors/ReduceProcessor.java     |    43 +
 .../streams/processors/StateQueryProcessor.java |    48 +
 .../streams/processors/StatefulProcessor.java   |    36 +
 .../processors/UpdateStateByKeyProcessor.java   |    49 +
 .../org/apache/storm/streams/tuple/Tuple10.java |   112 +
 .../org/apache/storm/streams/tuple/Tuple3.java  |    70 +
 .../org/apache/storm/streams/tuple/Tuple4.java  |    76 +
 .../org/apache/storm/streams/tuple/Tuple5.java  |    82 +
 .../org/apache/storm/streams/tuple/Tuple6.java  |    89 +
 .../org/apache/storm/streams/tuple/Tuple7.java  |    94 +
 .../org/apache/storm/streams/tuple/Tuple8.java  |   100 +
 .../org/apache/storm/streams/tuple/Tuple9.java  |   106 +
 .../storm/streams/windowing/BaseWindow.java     |    64 +
 .../storm/streams/windowing/SlidingWindows.java |   151 +
 .../streams/windowing/TumblingWindows.java      |   119 +
 .../apache/storm/streams/windowing/Window.java  |    70 +
 .../storm/task/GeneralTopologyContext.java      |   202 +
 .../src/jvm/org/apache/storm/task/IBolt.java    |    84 +
 .../org/apache/storm/task/IErrorReporter.java   |    22 +
 .../org/apache/storm/task/IMetricsContext.java  |    31 +
 .../org/apache/storm/task/IOutputCollector.java |    33 +
 .../org/apache/storm/task/OutputCollector.java  |   236 +
 .../jvm/org/apache/storm/task/ShellBolt.java    |   435 +
 .../org/apache/storm/task/TopologyContext.java  |   401 +
 .../storm/task/WorkerTopologyContext.java       |   106 +
 .../apache/storm/testing/AckFailDelegate.java   |    25 +
 .../apache/storm/testing/AckFailMapTracker.java |    52 +
 .../org/apache/storm/testing/AckTracker.java    |    52 +
 .../AlternateRackDNSToSwitchMapping.java        |    65 +
 .../apache/storm/testing/BatchNumberList.java   |    72 +
 .../apache/storm/testing/BatchProcessWord.java  |    39 +
 .../org/apache/storm/testing/BatchRepeatA.java  |    48 +
 .../org/apache/storm/testing/BoltTracker.java   |    42 +
 .../apache/storm/testing/CompletableSpout.java  |    39 +
 .../apache/storm/testing/CountingBatchBolt.java |    55 +
 .../storm/testing/CountingCommitBolt.java       |    57 +
 .../org/apache/storm/testing/FeederSpout.java   |   107 +
 .../org/apache/storm/testing/FixedTuple.java    |    43 +
 .../apache/storm/testing/FixedTupleSpout.java   |   185 +
 .../testing/ForwardingMetricsConsumer.java      |   100 +
 .../org/apache/storm/testing/IdentityBolt.java  |    42 +
 .../apache/storm/testing/IntegrationTest.java   |    38 +
 .../storm/testing/KeyedCountingBatchBolt.java   |    62 +
 .../testing/KeyedCountingCommitterBolt.java     |    24 +
 .../storm/testing/KeyedSummingBatchBolt.java    |    61 +
 .../storm/testing/MemoryTransactionalSpout.java |   182 +
 .../testing/MemoryTransactionalSpoutMeta.java   |    38 +
 .../apache/storm/testing/MkClusterParam.java    |    69 +
 .../org/apache/storm/testing/MkTupleParam.java  |    58 +
 .../org/apache/storm/testing/MockedSources.java |    68 +
 .../jvm/org/apache/storm/testing/NGrouping.java |    48 +
 .../storm/testing/NonRichBoltTracker.java       |    51 +
 .../testing/OpaqueMemoryTransactionalSpout.java |   188 +
 .../apache/storm/testing/PrepareBatchBolt.java  |    53 +
 .../storm/testing/PythonShellMetricsBolt.java   |    53 +
 .../storm/testing/PythonShellMetricsSpout.java  |    56 +
 .../testing/SingleUserSimpleTransport.java      |    37 +
 .../org/apache/storm/testing/SpoutTracker.java  |   111 +
 .../storm/testing/TestAggregatesCounter.java    |    63 +
 .../org/apache/storm/testing/TestConfBolt.java  |    62 +
 .../apache/storm/testing/TestEventLogSpout.java |   143 +
 .../storm/testing/TestEventOrderCheckBolt.java  |    76 +
 .../apache/storm/testing/TestGlobalCount.java   |    60 +
 .../apache/storm/testing/TestKryoDecorator.java |    31 +
 .../apache/storm/testing/TestPlannerBolt.java   |    45 +
 .../apache/storm/testing/TestPlannerSpout.java  |    86 +
 .../org/apache/storm/testing/TestSerObject.java |    37 +
 .../storm/testing/TestWordBytesCounter.java     |    27 +
 .../apache/storm/testing/TestWordCounter.java   |    65 +
 .../org/apache/storm/testing/TestWordSpout.java |    87 +
 .../jvm/org/apache/storm/testing/TmpPath.java   |    69 +
 .../apache/storm/testing/TupleCaptureBolt.java  |    83 +
 .../topology/BaseConfigurationDeclarer.java     |    83 +
 .../topology/BaseStatefulBoltExecutor.java      |   209 +
 .../storm/topology/BasicBoltExecutor.java       |    67 +
 .../storm/topology/BasicOutputCollector.java    |    72 +
 .../org/apache/storm/topology/BoltDeclarer.java |    26 +
 .../topology/CheckpointTupleForwarder.java      |   103 +
 .../ComponentConfigurationDeclarer.java         |    29 +
 .../apache/storm/topology/FailedException.java  |    36 +
 .../org/apache/storm/topology/IBasicBolt.java   |    33 +
 .../storm/topology/IBasicOutputCollector.java   |    29 +
 .../org/apache/storm/topology/IComponent.java   |    44 +
 .../org/apache/storm/topology/IRichBolt.java    |    29 +
 .../org/apache/storm/topology/IRichSpout.java   |    29 +
 .../apache/storm/topology/IRichStateSpout.java  |    25 +
 .../apache/storm/topology/IStatefulBolt.java    |    48 +
 .../storm/topology/IStatefulComponent.java      |    56 +
 .../storm/topology/IStatefulWindowedBolt.java   |    26 +
 .../apache/storm/topology/IWindowedBolt.java    |    51 +
 .../apache/storm/topology/InputDeclarer.java    |   184 +
 .../storm/topology/OutputFieldsDeclarer.java    |    32 +
 .../storm/topology/OutputFieldsGetter.java      |    53 +
 .../storm/topology/ReportedFailedException.java |    36 +
 .../apache/storm/topology/ResourceDeclarer.java |    28 +
 .../apache/storm/topology/SpoutDeclarer.java    |    22 +
 .../storm/topology/StatefulBoltExecutor.java    |   193 +
 .../topology/StatefulWindowedBoltExecutor.java  |   372 +
 .../apache/storm/topology/TopologyBuilder.java  |   587 +
 .../topology/TupleFieldTimestampExtractor.java  |    48 +
 .../storm/topology/WindowedBoltExecutor.java    |   363 +
 .../storm/topology/base/BaseBasicBolt.java      |    33 +
 .../storm/topology/base/BaseBatchBolt.java      |    24 +
 .../storm/topology/base/BaseComponent.java      |    28 +
 ...BaseOpaquePartitionedTransactionalSpout.java |    25 +
 .../base/BasePartitionedTransactionalSpout.java |    24 +
 .../storm/topology/base/BaseRichBolt.java       |    26 +
 .../storm/topology/base/BaseRichSpout.java      |    46 +
 .../storm/topology/base/BaseStatefulBolt.java   |    64 +
 .../topology/base/BaseStatefulWindowedBolt.java |   168 +
 .../base/BaseTickTupleAwareRichBolt.java        |    60 +
 .../topology/base/BaseTransactionalBolt.java    |    24 +
 .../topology/base/BaseTransactionalSpout.java   |    24 +
 .../storm/topology/base/BaseWindowedBolt.java   |   365 +
 .../apache/storm/transactional/ICommitter.java  |    26 +
 .../ICommitterTransactionalSpout.java           |    31 +
 .../transactional/ITransactionalSpout.java      |    94 +
 .../storm/transactional/TransactionAttempt.java |    61 +
 .../TransactionalSpoutBatchExecutor.java        |    96 +
 .../TransactionalSpoutCoordinator.java          |   214 +
 .../TransactionalTopologyBuilder.java           |   521 +
 .../IOpaquePartitionedTransactionalSpout.java   |    56 +
 .../IPartitionedTransactionalSpout.java         |    69 +
 ...uePartitionedTransactionalSpoutExecutor.java |   160 +
 .../PartitionedTransactionalSpoutExecutor.java  |   142 +
 .../state/RotatingTransactionalState.java       |   149 +
 .../state/TestTransactionalState.java           |    47 +
 .../transactional/state/TransactionalState.java |   172 +
 .../apache/storm/trident/JoinOutFieldsMode.java |    33 +
 .../jvm/org/apache/storm/trident/JoinType.java  |    30 +
 .../jvm/org/apache/storm/trident/Stream.java    |   897 +
 .../org/apache/storm/trident/TridentState.java  |    59 +
 .../apache/storm/trident/TridentTopology.java   |   986 +
 .../trident/drpc/ReturnResultsReducer.java      |   129 +
 .../fluent/ChainedAggregatorDeclarer.java       |   183 +
 .../fluent/ChainedFullAggregatorDeclarer.java   |    32 +
 .../ChainedPartitionAggregatorDeclarer.java     |    32 +
 .../trident/fluent/GlobalAggregationScheme.java |    26 +
 .../storm/trident/fluent/GroupedStream.java     |   174 +
 .../trident/fluent/IAggregatableStream.java     |    31 +
 .../fluent/IChainedAggregatorDeclarer.java      |    24 +
 .../storm/trident/fluent/UniqueIdGen.java       |    38 +
 .../storm/trident/graph/GraphGrouper.java       |   119 +
 .../org/apache/storm/trident/graph/Group.java   |   150 +
 .../storm/trident/operation/Aggregator.java     |    26 +
 .../storm/trident/operation/Assembly.java       |    45 +
 .../storm/trident/operation/BaseAggregator.java |    23 +
 .../storm/trident/operation/BaseFilter.java     |    23 +
 .../storm/trident/operation/BaseFunction.java   |    23 +
 .../trident/operation/BaseMultiReducer.java     |    33 +
 .../storm/trident/operation/BaseOperation.java  |    47 +
 .../trident/operation/CombinerAggregator.java   |    29 +
 .../storm/trident/operation/Consumer.java       |    35 +
 .../operation/DefaultResourceDeclarer.java      |    68 +
 .../storm/trident/operation/EachOperation.java  |    22 +
 .../apache/storm/trident/operation/Filter.java  |    48 +
 .../trident/operation/FlatMapFunction.java      |    37 +
 .../storm/trident/operation/Function.java       |    92 +
 .../trident/operation/GroupedMultiReducer.java  |    31 +
 .../trident/operation/ITridentResource.java     |    32 +
 .../storm/trident/operation/MapFunction.java    |    36 +
 .../storm/trident/operation/MultiReducer.java   |    31 +
 .../storm/trident/operation/Operation.java      |    46 +
 .../OperationAwareFlatMapFunction.java          |    24 +
 .../operation/OperationAwareMapFunction.java    |    24 +
 .../trident/operation/ReducerAggregator.java    |    26 +
 .../trident/operation/TridentCollector.java     |    52 +
 .../operation/TridentMultiReducerContext.java   |    36 +
 .../operation/TridentOperationContext.java      |    65 +
 .../operation/builtin/ComparisonAggregator.java |    91 +
 .../storm/trident/operation/builtin/Count.java  |    41 +
 .../storm/trident/operation/builtin/Debug.java  |    62 +
 .../storm/trident/operation/builtin/Equals.java |    38 +
 .../trident/operation/builtin/FilterNull.java   |    35 +
 .../storm/trident/operation/builtin/FirstN.java |   130 +
 .../storm/trident/operation/builtin/MapGet.java |    38 +
 .../storm/trident/operation/builtin/Max.java    |    37 +
 .../operation/builtin/MaxWithComparator.java    |    51 +
 .../storm/trident/operation/builtin/Min.java    |    36 +
 .../operation/builtin/MinWithComparator.java    |    51 +
 .../storm/trident/operation/builtin/Negate.java |    76 +
 .../trident/operation/builtin/SnapshotGet.java  |    44 +
 .../storm/trident/operation/builtin/Sum.java    |    42 +
 .../operation/builtin/TupleCollectionGet.java   |    46 +
 .../operation/impl/CaptureCollector.java        |    42 +
 .../operation/impl/ChainedAggregatorImpl.java   |   113 +
 .../trident/operation/impl/ChainedResult.java   |    53 +
 .../operation/impl/CombinerAggStateUpdater.java |    56 +
 .../impl/CombinerAggregatorCombineImpl.java     |    61 +
 .../impl/CombinerAggregatorInitImpl.java        |    49 +
 .../operation/impl/ConsumerExecutor.java        |    38 +
 .../trident/operation/impl/FilterExecutor.java  |    53 +
 .../operation/impl/FlatMapFunctionExecutor.java |    64 +
 .../operation/impl/GlobalBatchToPartition.java  |    29 +
 .../trident/operation/impl/GroupCollector.java  |    48 +
 .../operation/impl/GroupedAggregator.java       |    96 +
 .../impl/GroupedMultiReducerExecutor.java       |    95 +
 .../operation/impl/IdentityMultiReducer.java    |    51 +
 .../impl/IndexHashBatchToPartition.java         |    29 +
 .../storm/trident/operation/impl/JoinState.java |    39 +
 .../operation/impl/JoinerMultiReducer.java      |   142 +
 .../operation/impl/MapFunctionExecutor.java     |    62 +
 ...PreservingFieldsOrderJoinerMultiReducer.java |   175 +
 .../operation/impl/ReducerAggStateUpdater.java  |    53 +
 .../operation/impl/ReducerAggregatorImpl.java   |    56 +
 .../storm/trident/operation/impl/Result.java    |    27 +
 .../operation/impl/SingleEmitAggregator.java    |    95 +
 .../trident/operation/impl/TrueFilter.java      |    40 +
 .../storm/trident/partition/GlobalGrouping.java |    42 +
 .../trident/partition/IdentityGrouping.java     |    59 +
 .../trident/partition/IndexHashGrouping.java    |    55 +
 .../storm/trident/planner/BridgeReceiver.java   |    38 +
 .../org/apache/storm/trident/planner/Node.java  |    69 +
 .../storm/trident/planner/NodeStateInfo.java    |    31 +
 .../storm/trident/planner/PartitionNode.java    |    50 +
 .../storm/trident/planner/ProcessorContext.java |    29 +
 .../storm/trident/planner/ProcessorNode.java    |    38 +
 .../apache/storm/trident/planner/SpoutNode.java |    39 +
 .../storm/trident/planner/SubtopologyBolt.java  |   246 +
 .../storm/trident/planner/TridentProcessor.java |    40 +
 .../storm/trident/planner/TupleReceiver.java    |    27 +
 .../planner/processor/AggregateProcessor.java   |    84 +
 .../planner/processor/AppendCollector.java      |    62 +
 .../planner/processor/EachProcessor.java        |    80 +
 .../planner/processor/FreshCollector.java       |    59 +
 .../trident/planner/processor/MapProcessor.java |    87 +
 .../processor/MultiReducerProcessor.java        |    93 +
 .../processor/PartitionPersistProcessor.java    |   107 +
 .../planner/processor/ProjectedProcessor.java   |    73 +
 .../planner/processor/StateQueryProcessor.java  |   107 +
 .../planner/processor/TridentContext.java       |    76 +
 .../storm/trident/spout/BatchSpoutExecutor.java |    92 +
 .../apache/storm/trident/spout/IBatchID.java    |    24 +
 .../apache/storm/trident/spout/IBatchSpout.java |    33 +
 .../trident/spout/ICommitterTridentSpout.java   |    31 +
 .../spout/IOpaquePartitionedTridentSpout.java   |    87 +
 .../trident/spout/IPartitionedTridentSpout.java |    77 +
 .../storm/trident/spout/ISpoutPartition.java    |    25 +
 .../storm/trident/spout/ITridentDataSource.java |    26 +
 .../storm/trident/spout/ITridentSpout.java      |   125 +
 .../OpaquePartitionedTridentSpoutExecutor.java  |   234 +
 .../spout/PartitionedTridentSpoutExecutor.java  |   188 +
 .../trident/spout/RichSpoutBatchExecutor.java   |   204 +
 .../storm/trident/spout/RichSpoutBatchId.java   |    49 +
 .../spout/RichSpoutBatchIdSerializer.java       |    38 +
 .../trident/spout/RichSpoutBatchTriggerer.java  |   183 +
 .../trident/spout/TridentSpoutCoordinator.java  |    94 +
 .../trident/spout/TridentSpoutExecutor.java     |   138 +
 .../storm/trident/state/BaseQueryFunction.java  |    25 +
 .../storm/trident/state/BaseStateUpdater.java   |    25 +
 .../trident/state/CombinerValueUpdater.java     |    45 +
 .../storm/trident/state/ITupleCollection.java   |    26 +
 .../state/JSONNonTransactionalSerializer.java   |    45 +
 .../trident/state/JSONOpaqueSerializer.java     |    52 +
 .../state/JSONTransactionalSerializer.java      |    50 +
 .../apache/storm/trident/state/OpaqueValue.java |    75 +
 .../storm/trident/state/QueryFunction.java      |    28 +
 .../storm/trident/state/ReadOnlyState.java      |    31 +
 .../trident/state/ReducerValueUpdater.java      |    41 +
 .../apache/storm/trident/state/Serializer.java  |    26 +
 .../org/apache/storm/trident/state/State.java   |    39 +
 .../storm/trident/state/StateFactory.java       |    26 +
 .../apache/storm/trident/state/StateSpec.java   |    30 +
 .../apache/storm/trident/state/StateType.java   |    25 +
 .../storm/trident/state/StateUpdater.java       |    33 +
 .../storm/trident/state/TransactionalValue.java |    44 +
 .../storm/trident/state/ValueUpdater.java       |    23 +
 .../trident/state/map/CachedBatchReadsMap.java  |    80 +
 .../storm/trident/state/map/CachedMap.java      |    78 +
 .../storm/trident/state/map/IBackingMap.java    |    26 +
 .../state/map/MapCombinerAggStateUpdater.java   |    84 +
 .../state/map/MapReducerAggStateUpdater.java    |    91 +
 .../storm/trident/state/map/MapState.java       |    26 +
 .../state/map/MicroBatchIBackingMap.java        |    85 +
 .../trident/state/map/NonTransactionalMap.java  |    67 +
 .../storm/trident/state/map/OpaqueMap.java      |   124 +
 .../trident/state/map/ReadOnlyMapState.java     |    26 +
 .../trident/state/map/RemovableMapState.java    |    25 +
 .../trident/state/map/SnapshottableMap.java     |    76 +
 .../trident/state/map/TransactionalMap.java     |   109 +
 .../state/snapshot/ReadOnlySnapshottable.java   |    24 +
 .../trident/state/snapshot/Snapshottable.java   |    27 +
 .../trident/testing/CountAsAggregator.java      |    47 +
 .../storm/trident/testing/FeederBatchSpout.java |   185 +
 .../testing/FeederCommitterBatchSpout.java      |    96 +
 .../storm/trident/testing/FixedBatchSpout.java  |    97 +
 .../apache/storm/trident/testing/IFeeder.java   |    23 +
 .../trident/testing/LRUMemoryMapState.java      |   154 +
 .../storm/trident/testing/MemoryBackingMap.java |    47 +
 .../storm/trident/testing/MemoryMapState.java   |   176 +
 .../org/apache/storm/trident/testing/Split.java |    36 +
 .../storm/trident/testing/StringLength.java     |    32 +
 .../storm/trident/testing/TrueFilter.java       |    30 +
 .../storm/trident/testing/TuplifyArgs.java      |    41 +
 .../storm/trident/topology/BatchInfo.java       |    33 +
 .../trident/topology/ITridentBatchBolt.java     |    32 +
 .../topology/MasterBatchCoordinator.java        |   316 +
 .../trident/topology/TransactionAttempt.java    |    66 +
 .../trident/topology/TridentBoltExecutor.java   |   440 +
 .../topology/TridentTopologyBuilder.java        |   751 +
 .../state/RotatingTransactionalState.java       |   186 +
 .../topology/state/TestTransactionalState.java  |    47 +
 .../topology/state/TransactionalState.java      |   192 +
 .../apache/storm/trident/tuple/ComboList.java   |    92 +
 .../apache/storm/trident/tuple/ConsList.java    |    60 +
 .../storm/trident/tuple/TridentTuple.java       |    34 +
 .../storm/trident/tuple/TridentTupleView.java   |   361 +
 .../storm/trident/tuple/ValuePointer.java       |    60 +
 .../storm/trident/util/ErrorEdgeFactory.java    |    28 +
 .../apache/storm/trident/util/IndexedEdge.java  |    50 +
 .../org/apache/storm/trident/util/LRUMap.java   |    35 +
 .../apache/storm/trident/util/TridentUtils.java |   115 +
 .../windowing/AbstractTridentWindowManager.java |   238 +
 .../windowing/ITridentWindowManager.java        |    59 +
 .../windowing/InMemoryTridentWindowManager.java |    72 +
 .../trident/windowing/InMemoryWindowsStore.java |   200 +
 .../windowing/InMemoryWindowsStoreFactory.java  |    44 +
 .../StoreBasedTridentWindowManager.java         |   217 +
 .../trident/windowing/TridentBatchTuple.java    |    42 +
 .../trident/windowing/WindowKryoSerializer.java |    87 +
 .../windowing/WindowTridentProcessor.java       |   265 +
 .../storm/trident/windowing/WindowsState.java   |    52 +
 .../trident/windowing/WindowsStateFactory.java  |    40 +
 .../trident/windowing/WindowsStateUpdater.java  |    81 +
 .../storm/trident/windowing/WindowsStore.java   |    79 +
 .../trident/windowing/WindowsStoreFactory.java  |    40 +
 .../windowing/config/BaseWindowConfig.java      |    48 +
 .../windowing/config/SlidingCountWindow.java    |    43 +
 .../windowing/config/SlidingDurationWindow.java |    44 +
 .../windowing/config/TumblingCountWindow.java   |    43 +
 .../config/TumblingDurationWindow.java          |    42 +
 .../trident/windowing/config/WindowConfig.java  |    57 +
 .../windowing/strategy/BaseWindowStrategy.java  |    32 +
 .../strategy/SlidingCountWindowStrategy.java    |    59 +
 .../strategy/SlidingDurationWindowStrategy.java |    60 +
 .../strategy/TumblingCountWindowStrategy.java   |    60 +
 .../TumblingDurationWindowStrategy.java         |    60 +
 .../windowing/strategy/WindowStrategy.java      |    45 +
 .../org/apache/storm/tuple/AddressedTuple.java  |    48 +
 .../src/jvm/org/apache/storm/tuple/Fields.java  |   142 +
 .../src/jvm/org/apache/storm/tuple/ITuple.java  |   214 +
 .../jvm/org/apache/storm/tuple/MessageId.java   |    93 +
 .../src/jvm/org/apache/storm/tuple/Tuple.java   |    68 +
 .../jvm/org/apache/storm/tuple/TupleImpl.java   |   356 +
 .../src/jvm/org/apache/storm/tuple/Values.java  |    37 +
 .../jvm/org/apache/storm/utils/BatchHelper.java |    92 +
 .../storm/utils/BufferFileInputStream.java      |    54 +
 .../apache/storm/utils/CRC32OutputStream.java   |    44 +
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   365 +
 .../org/apache/storm/utils/CuratorUtils.java    |   115 +
 .../jvm/org/apache/storm/utils/DRPCClient.java  |    66 +
 .../utils/DisruptorBackpressureCallback.java    |    27 +
 .../org/apache/storm/utils/DisruptorQueue.java  |   589 +
 .../storm/utils/ExtendedThreadPoolExecutor.java |    67 +
 .../jvm/org/apache/storm/utils/IPredicate.java  |    27 +
 .../storm/utils/IndifferentAccessMap.java       |   177 +
 .../org/apache/storm/utils/InprocMessaging.java |   100 +
 .../storm/utils/KeyedRoundRobinQueue.java       |    68 +
 .../org/apache/storm/utils/ListDelegate.java    |   156 +
 .../jvm/org/apache/storm/utils/LocalState.java  |   277 +
 .../jvm/org/apache/storm/utils/MutableInt.java  |    43 +
 .../jvm/org/apache/storm/utils/MutableLong.java |    43 +
 .../org/apache/storm/utils/MutableObject.java   |    38 +
 .../org/apache/storm/utils/NimbusClient.java    |   139 +
 .../utils/NimbusLeaderNotFoundException.java    |    41 +
 .../org/apache/storm/utils/ObjectReader.java    |   119 +
 .../org/apache/storm/utils/ReflectionUtils.java |    83 +
 .../storm/utils/RegisteredGlobalState.java      |    62 +
 .../jvm/org/apache/storm/utils/RotatingMap.java |   128 +
 .../org/apache/storm/utils/ServiceRegistry.java |    47 +
 .../storm/utils/ShellBoltMessageQueue.java      |   121 +
 .../org/apache/storm/utils/ShellProcess.java    |   215 +
 .../jvm/org/apache/storm/utils/ShellUtils.java  |   505 +
 .../StormBoundedExponentialBackoffRetry.java    |    79 +
 .../apache/storm/utils/ThriftTopologyUtils.java |    70 +
 .../src/jvm/org/apache/storm/utils/Time.java    |   235 +
 .../org/apache/storm/utils/TimeCacheMap.java    |   125 +
 .../org/apache/storm/utils/TransferDrainer.java |   140 +
 .../jvm/org/apache/storm/utils/TupleUtils.java  |    67 +
 .../src/jvm/org/apache/storm/utils/Utils.java   |  1273 +
 .../jvm/org/apache/storm/utils/VersionInfo.java |   131 +
 .../org/apache/storm/utils/VersionedStore.java  |   188 +
 .../storm/utils/WindowedTimeThrottler.java      |    51 +
 .../storm/utils/WorkerBackpressureCallback.java |    26 +
 .../storm/utils/WorkerBackpressureThread.java   |    80 +
 .../org/apache/storm/utils/WritableUtils.java   |   375 +
 .../apache/storm/utils/ZookeeperAuthInfo.java   |    53 +
 .../storm/utils/ZookeeperServerCnxnFactory.java |    84 +
 .../storm/validation/ConfigValidation.java      |   716 +
 .../validation/ConfigValidationAnnotations.java |   218 +
 .../storm/validation/ConfigValidationUtils.java |   175 +
 .../storm/windowing/CountEvictionPolicy.java    |    81 +
 .../storm/windowing/CountTriggerPolicy.java     |    76 +
 .../storm/windowing/DefaultEvictionContext.java |    53 +
 .../jvm/org/apache/storm/windowing/Event.java   |    49 +
 .../org/apache/storm/windowing/EventImpl.java   |    51 +
 .../apache/storm/windowing/EvictionContext.java |    47 +
 .../apache/storm/windowing/EvictionPolicy.java  |    82 +
 .../storm/windowing/TimeEvictionPolicy.java     |    72 +
 .../storm/windowing/TimeTriggerPolicy.java      |   132 +
 .../storm/windowing/TimestampExtractor.java     |    35 +
 .../apache/storm/windowing/TriggerHandler.java  |    31 +
 .../apache/storm/windowing/TriggerPolicy.java   |    49 +
 .../org/apache/storm/windowing/TupleWindow.java |    26 +
 .../apache/storm/windowing/TupleWindowImpl.java |    93 +
 .../apache/storm/windowing/WaterMarkEvent.java  |    38 +
 .../windowing/WaterMarkEventGenerator.java      |   121 +
 .../windowing/WatermarkCountEvictionPolicy.java |    68 +
 .../windowing/WatermarkCountTriggerPolicy.java  |    91 +
 .../windowing/WatermarkTimeEvictionPolicy.java  |    77 +
 .../windowing/WatermarkTimeTriggerPolicy.java   |   126 +
 .../jvm/org/apache/storm/windowing/Window.java  |    55 +
 .../windowing/WindowLifecycleListener.java      |    43 +
 .../apache/storm/windowing/WindowManager.java   |   289 +
 .../apache/storm/zookeeper/ClientZookeeper.java |   345 +
 .../apache/storm/zookeeper/ZkEventTypes.java    |    43 +
 .../apache/storm/zookeeper/ZkKeeperStates.java  |    41 +
 storm-client/src/py/__init__.py                 |    18 +
 storm-client/src/py/storm/DistributedRPC-remote |   112 +
 storm-client/src/py/storm/DistributedRPC.py     |   311 +
 .../py/storm/DistributedRPCInvocations-remote   |   133 +
 .../src/py/storm/DistributedRPCInvocations.py   |   886 +
 storm-client/src/py/storm/Nimbus-remote         |   420 +
 storm-client/src/py/storm/Nimbus.py             |  9915 ++++
 storm-client/src/py/storm/__init__.py           |    19 +
 storm-client/src/py/storm/constants.py          |    29 +
 storm-client/src/py/storm/ttypes.py             | 11695 ++++
 storm-client/src/py_license_header.txt          |    18 +
 .../storm-client-version-info.properties        |    24 +
 storm-client/src/storm.thrift                   |   775 +
 .../storm/PaceMakerStateStorageFactoryTest.java |   153 +
 .../org/apache/storm/TestConfigValidate.java    |   726 +
 .../storm/blobstore/ClientBlobStoreTest.java    |   179 +
 .../jvm/org/apache/storm/bolt/TestJoinBolt.java |   357 +
 .../cluster/StormClusterStateImplTest.java      |   116 +
 .../daemon/worker/LogConfigManagerTest.java     |   202 +
 .../apache/storm/daemon/worker/WorkerTest.java  |    39 +
 .../DependencyPropertiesParserTest.java         |    72 +
 .../dependency/DependencyUploaderTest.java      |   296 +
 .../storm/executor/error/ReportErrorTest.java   |    74 +
 .../storm/grouping/PartialKeyGroupingTest.java  |    66 +
 .../storm/grouping/ShuffleGroupingTest.java     |   147 +
 .../metric/filter/FilterByMetricNameTest.java   |    95 +
 .../metric/internal/CountStatAndMetricTest.java |    86 +
 .../internal/LatencyStatAndMetricTest.java      |    83 +
 .../storm/metric/internal/RateTrackerTest.java  |    94 +
 .../metric/util/DataPointExpanderTest.java      |   109 +
 .../storm/security/auth/AuthUtilsTest.java      |   240 +
 .../storm/security/auth/AuthUtilsTestMock.java  |    82 +
 .../apache/storm/security/auth/AutoSSLTest.java |   136 +
 .../auth/DefaultHttpCredentialsPluginTest.java  |    90 +
 .../storm/security/auth/ReqContextTest.java     |    87 +
 .../security/auth/SaslTransportPluginTest.java  |    49 +
 .../storm/security/auth/ThriftClientTest.java   |    68 +
 .../storm/security/auth/ThriftServerTest.java   |    38 +
 .../authorizer/DRPCSimpleACLAuthorizerTest.java |   163 +
 .../BlowfishTupleSerializerTest.java            |    86 +
 .../GzipBridgeSerializationDelegateTest.java    |    82 +
 ...ipBridgeThriftSerializationDelegateTest.java |    71 +
 .../serialization/SerializationFactoryTest.java |    61 +
 .../ThriftBridgeSerializationDelegateTest.java  |    60 +
 .../apache/storm/spout/CheckpointSpoutTest.java |   262 +
 .../storm/state/InMemoryKeyValueStateTest.java  |    99 +
 .../apache/storm/streams/ProcessorBoltTest.java |   174 +
 .../streams/StatefulProcessorBoltTest.java      |   111 +
 .../apache/storm/streams/StreamBuilderTest.java |   268 +
 .../streams/WindowedProcessorBoltTest.java      |   110 +
 .../streams/processors/JoinProcessorTest.java   |   125 +
 .../topology/StatefulBoltExecutorTest.java      |   178 +
 .../StatefulWindowedBoltExecutorTest.java       |   131 +
 .../storm/topology/TopologyBuilderTest.java     |   118 +
 .../topology/WindowedBoltExecutorTest.java      |   230 +
 .../storm/trident/TestTridentTopology.java      |    76 +
 .../storm/trident/TridentWindowingTest.java     |   105 +
 .../jvm/org/apache/storm/tuple/FieldsTest.java  |   126 +
 .../apache/storm/utils/CuratorUtilsTest.java    |   108 +
 .../utils/DisruptorQueueBackpressureTest.java   |   110 +
 .../apache/storm/utils/DisruptorQueueTest.java  |   187 +
 .../apache/storm/utils/MockTupleHelpers.java    |    40 +
 .../storm/utils/ShellBoltMessageQueueTest.java  |    85 +
 ...StormBoundedExponentialBackoffRetryTest.java |   101 +
 .../storm/utils/ThriftTopologyUtilsTest.java    |   106 +
 .../utils/ThrowableNestedCauseMatcher.java      |    44 +
 .../jvm/org/apache/storm/utils/TimeTest.java    |    95 +
 .../jvm/org/apache/storm/utils/UtilsTest.java   |   141 +
 .../utils/WorkerBackpressureThreadTest.java     |    50 +
 .../utils/ZookeeperServerCnxnFactoryTest.java   |    39 +
 .../windowing/WaterMarkEventGeneratorTest.java  |   118 +
 .../storm/windowing/WindowManagerTest.java      |   570 +
 .../drpc-simple-acl-test-scenario.yaml          |    28 +
 storm-clojure/pom.xml                           |     6 +
 .../src/clj/org/apache/storm/testing.clj        |     2 +-
 .../src/clj/org/apache/storm/thrift.clj         |    23 +-
 storm-clojure/src/test/clj/clojure_test.clj     |    40 +-
 storm-core/pom.xml                              |   242 +-
 .../apache/storm/command/shell_submission.clj   |     4 +-
 storm-core/src/clj/org/apache/storm/config.clj  |     6 +-
 .../clj/org/apache/storm/daemon/logviewer.clj   |    68 +-
 .../clj/org/apache/storm/internal/thrift.clj    |    17 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |    84 +-
 storm-core/src/clj/org/apache/storm/util.clj    |    34 +-
 .../src/dev/drpc-simple-acl-test-scenario.yaml  |    28 -
 storm-core/src/genthrift.sh                     |    33 -
 storm-core/src/java_license_header.txt          |    17 -
 storm-core/src/jvm/org/apache/storm/Config.java |  2593 -
 .../src/jvm/org/apache/storm/Constants.java     |    60 -
 .../org/apache/storm/ICredentialsListener.java  |    32 -
 .../src/jvm/org/apache/storm/ILocalCluster.java |   202 -
 .../src/jvm/org/apache/storm/ILocalDRPC.java    |    36 -
 .../jvm/org/apache/storm/ISubmitterHook.java    |    31 -
 .../src/jvm/org/apache/storm/LocalCluster.java  |   834 -
 .../src/jvm/org/apache/storm/LocalDRPC.java     |    90 -
 .../src/jvm/org/apache/storm/LogWriter.java     |    83 -
 .../jvm/org/apache/storm/ProcessSimulator.java  |    97 -
 .../jvm/org/apache/storm/StormSubmitter.java    |   602 -
 .../src/jvm/org/apache/storm/StormTimer.java    |   245 -
 .../src/jvm/org/apache/storm/Testing.java       |   716 -
 storm-core/src/jvm/org/apache/storm/Thrift.java |   351 -
 .../storm/annotation/InterfaceStability.java    |    54 -
 .../storm/blobstore/AtomicOutputStream.java     |    32 -
 .../storm/blobstore/BlobKeySequenceInfo.java    |    40 -
 .../org/apache/storm/blobstore/BlobStore.java   |   490 -
 .../storm/blobstore/BlobStoreAclHandler.java    |   409 -
 .../apache/storm/blobstore/BlobStoreFile.java   |    50 -
 .../apache/storm/blobstore/BlobStoreUtils.java  |   288 -
 .../storm/blobstore/BlobSynchronizer.java       |   131 -
 .../apache/storm/blobstore/ClientBlobStore.java |   201 -
 .../storm/blobstore/FileBlobStoreImpl.java      |   268 -
 .../storm/blobstore/InputStreamWithMeta.java    |    26 -
 .../org/apache/storm/blobstore/KeyFilter.java   |    22 -
 .../storm/blobstore/KeySequenceNumber.java      |   238 -
 .../storm/blobstore/LocalFsBlobStore.java       |   350 -
 .../storm/blobstore/LocalFsBlobStoreFile.java   |   159 -
 .../apache/storm/blobstore/NimbusBlobStore.java |   428 -
 .../src/jvm/org/apache/storm/bolt/JoinBolt.java |   595 -
 .../storm/callback/DefaultWatcherCallBack.java  |    35 -
 .../apache/storm/callback/WatcherCallBack.java  |    24 -
 .../storm/callback/ZKStateChangedCallback.java  |    25 -
 .../org/apache/storm/clojure/ClojureBolt.java   |   119 -
 .../org/apache/storm/clojure/ClojureSpout.java  |   153 -
 .../org/apache/storm/clojure/RichShellBolt.java |    51 -
 .../apache/storm/clojure/RichShellSpout.java    |    51 -
 .../storm/cluster/ClusterStateContext.java      |    41 -
 .../storm/cluster/ClusterStateListener.java     |    22 -
 .../org/apache/storm/cluster/ClusterUtils.java  |   244 -
 .../apache/storm/cluster/ConnectionState.java   |    24 -
 .../org/apache/storm/cluster/DaemonType.java    |    27 -
 .../org/apache/storm/cluster/ExecutorBeat.java  |    44 -
 .../org/apache/storm/cluster/IStateStorage.java |   219 -
 .../storm/cluster/IStormClusterState.java       |   198 -
 .../storm/cluster/PaceMakerStateStorage.java    |   266 -
 .../cluster/PaceMakerStateStorageFactory.java   |    37 -
 .../storm/cluster/StateStorageFactory.java      |    28 -
 .../storm/cluster/StormClusterStateImpl.java    |   724 -
 .../org/apache/storm/cluster/VersionedData.java |    36 -
 .../apache/storm/cluster/ZKStateStorage.java    |   252 -
 .../storm/cluster/ZKStateStorageFactory.java    |    36 -
 .../org/apache/storm/command/AdminCommands.java |    14 +-
 .../jvm/org/apache/storm/command/Blobstore.java |     4 +-
 .../org/apache/storm/command/ConfigValue.java   |     4 +-
 .../org/apache/storm/command/DevZookeeper.java  |    10 +-
 .../jvm/org/apache/storm/command/GetErrors.java |     2 +-
 .../org/apache/storm/command/HealthCheck.java   |   102 +-
 .../org/apache/storm/command/Heartbeats.java    |     4 +-
 .../org/apache/storm/command/KillWorkers.java   |     4 +-
 .../org/apache/storm/command/SetLogLevel.java   |     2 +-
 .../container/ResourceIsolationInterface.java   |    76 -
 .../storm/container/cgroup/CgroupCenter.java    |   216 -
 .../storm/container/cgroup/CgroupCommon.java    |   270 -
 .../container/cgroup/CgroupCommonOperation.java |    81 -
 .../container/cgroup/CgroupCoreFactory.java     |    72 -
 .../storm/container/cgroup/CgroupManager.java   |   223 -
 .../storm/container/cgroup/CgroupOperation.java |    79 -
 .../storm/container/cgroup/CgroupUtils.java     |   118 -
 .../apache/storm/container/cgroup/Device.java   |    75 -
 .../storm/container/cgroup/Hierarchy.java       |   130 -
 .../storm/container/cgroup/SubSystem.java       |    81 -
 .../storm/container/cgroup/SubSystemType.java   |    36 -
 .../storm/container/cgroup/SystemOperation.java |    76 -
 .../storm/container/cgroup/core/BlkioCore.java  |   213 -
 .../storm/container/cgroup/core/CgroupCore.java |    26 -
 .../storm/container/cgroup/core/CpuCore.java    |   135 -
 .../container/cgroup/core/CpuacctCore.java      |    71 -
 .../storm/container/cgroup/core/CpusetCore.java |   209 -
 .../container/cgroup/core/DevicesCore.java      |   189 -
 .../container/cgroup/core/FreezerCore.java      |    66 -
 .../storm/container/cgroup/core/MemoryCore.java |   188 -
 .../storm/container/cgroup/core/NetClsCore.java |    69 -
 .../container/cgroup/core/NetPrioCore.java      |    65 -
 .../storm/coordination/BatchBoltExecutor.java   |   108 -
 .../coordination/BatchOutputCollector.java      |    46 -
 .../coordination/BatchOutputCollectorImpl.java  |    53 -
 .../coordination/BatchSubtopologyBuilder.java   |   447 -
 .../storm/coordination/CoordinatedBolt.java     |   386 -
 .../apache/storm/coordination/IBatchBolt.java   |    30 -
 .../src/jvm/org/apache/storm/daemon/Acker.java  |   140 -
 .../daemon/ClientJarTransformerRunner.java      |    12 +-
 .../org/apache/storm/daemon/DaemonCommon.java   |    22 -
 .../apache/storm/daemon/DirectoryCleaner.java   |     3 -
 .../org/apache/storm/daemon/GrouperFactory.java |   244 -
 .../org/apache/storm/daemon/Shutdownable.java   |    22 -
 .../org/apache/storm/daemon/StormCommon.java    |   596 -
 .../src/jvm/org/apache/storm/daemon/Task.java   |   247 -
 .../daemon/drpc/BlockingOutstandingRequest.java |    67 -
 .../jvm/org/apache/storm/daemon/drpc/DRPC.java  |   215 -
 .../apache/storm/daemon/drpc/DRPCThrift.java    |    58 -
 .../storm/daemon/drpc/OutstandingRequest.java   |    58 -
 .../storm/daemon/drpc/RequestFactory.java       |    24 -
 .../daemon/metrics/BuiltinBoltMetrics.java      |    78 -
 .../storm/daemon/metrics/BuiltinMetrics.java    |    33 -
 .../daemon/metrics/BuiltinMetricsUtil.java      |    81 -
 .../daemon/metrics/BuiltinSpoutMetrics.java     |    64 -
 .../storm/daemon/metrics/MetricsUtils.java      |   108 -
 .../daemon/metrics/SpoutThrottlingMetrics.java  |    57 -
 .../reporters/ConsolePreparableReporter.java    |    76 -
 .../reporters/CsvPreparableReporter.java        |    80 -
 .../reporters/JmxPreparableReporter.java        |    70 -
 .../metrics/reporters/PreparableReporter.java   |    32 -
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |  3805 --
 .../storm/daemon/nimbus/TopologyActions.java    |    31 -
 .../storm/daemon/nimbus/TopologyResources.java  |    63 -
 .../daemon/nimbus/TopologyStateTransition.java  |    27 -
 .../storm/daemon/supervisor/AdvancedFSOps.java  |   363 -
 .../storm/daemon/supervisor/BasicContainer.java |   719 -
 .../supervisor/BasicContainerLauncher.java      |    62 -
 .../storm/daemon/supervisor/Container.java      |   565 -
 .../daemon/supervisor/ContainerLauncher.java    |   104 -
 .../supervisor/ContainerRecoveryException.java  |    29 -
 .../DefaultUncaughtExceptionHandler.java        |    31 -
 .../supervisor/EventManagerPushCallback.java    |    37 -
 .../daemon/supervisor/ExitCodeCallback.java     |    30 -
 .../storm/daemon/supervisor/Killable.java       |    50 -
 .../storm/daemon/supervisor/LocalContainer.java |    94 -
 .../supervisor/LocalContainerLauncher.java      |    60 -
 .../daemon/supervisor/ReadClusterState.java     |   358 -
 .../daemon/supervisor/RunAsUserContainer.java   |   100 -
 .../supervisor/RunAsUserContainerLauncher.java  |    60 -
 .../apache/storm/daemon/supervisor/Slot.java    |   785 -
 .../daemon/supervisor/StandaloneSupervisor.java |    91 -
 .../storm/daemon/supervisor/Supervisor.java     |   359 -
 .../daemon/supervisor/SupervisorUtils.java      |   321 -
 .../apache/storm/daemon/supervisor/UniFunc.java |    22 -
 .../supervisor/timer/SupervisorHealthCheck.java |    41 -
 .../supervisor/timer/SupervisorHeartbeat.java   |    87 -
 .../daemon/supervisor/timer/UpdateBlobs.java    |   105 -
 .../storm/daemon/worker/LogConfigManager.java   |   156 -
 .../org/apache/storm/daemon/worker/Worker.java  |   458 -
 .../apache/storm/daemon/worker/WorkerState.java |   691 -
 .../dependency/DependencyPropertiesParser.java  |    61 -
 .../storm/dependency/DependencyUploader.java    |   168 -
 .../dependency/FileNotAvailableException.java   |    33 -
 .../storm/drpc/DRPCInvocationsClient.java       |   130 -
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |   286 -
 .../jvm/org/apache/storm/drpc/JoinResult.java   |    75 -
 .../org/apache/storm/drpc/KeyedFairBolt.java    |    93 -
 .../storm/drpc/LinearDRPCInputDeclarer.java     |    52 -
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |   393 -
 .../org/apache/storm/drpc/PrepareRequest.java   |    59 -
 .../org/apache/storm/drpc/ReturnResults.java    |   137 -
 .../org/apache/storm/event/EventManager.java    |    24 -
 .../org/apache/storm/event/EventManagerImp.java |   100 -
 .../jvm/org/apache/storm/executor/Executor.java |   563 -
 .../apache/storm/executor/ExecutorShutdown.java |   114 -
 .../apache/storm/executor/ExecutorTransfer.java |    88 -
 .../apache/storm/executor/IRunningExecutor.java |    31 -
 .../apache/storm/executor/LocalExecutor.java    |    56 -
 .../org/apache/storm/executor/TupleInfo.java    |    90 -
 .../storm/executor/bolt/BoltExecutor.java       |   139 -
 .../executor/bolt/BoltOutputCollectorImpl.java  |   171 -
 .../storm/executor/error/IReportError.java      |    22 -
 .../storm/executor/error/ReportError.java       |    76 -
 .../storm/executor/error/ReportErrorAndDie.java |    49 -
 .../storm/executor/spout/SpoutExecutor.java     |   257 -
 .../spout/SpoutOutputCollectorImpl.java         |   147 -
 .../apache/storm/generated/AccessControl.java   |   627 -
 .../storm/generated/AccessControlType.java      |    62 -
 .../storm/generated/AlreadyAliveException.java  |   406 -
 .../org/apache/storm/generated/Assignment.java  |  1159 -
 .../storm/generated/AuthorizationException.java |   406 -
 .../storm/generated/BeginDownloadResult.java    |   608 -
 .../jvm/org/apache/storm/generated/Bolt.java    |   514 -
 .../storm/generated/BoltAggregateStats.java     |   704 -
 .../org/apache/storm/generated/BoltStats.java   |  1390 -
 .../apache/storm/generated/ClusterSummary.java  |   879 -
 .../storm/generated/ClusterWorkerHeartbeat.java |   768 -
 .../storm/generated/CommonAggregateStats.java   |  1060 -
 .../generated/ComponentAggregateStats.java      |   752 -
 .../apache/storm/generated/ComponentCommon.java |   852 -
 .../apache/storm/generated/ComponentObject.java |   462 -
 .../storm/generated/ComponentPageInfo.java      |  2352 -
 .../apache/storm/generated/ComponentType.java   |    62 -
 .../org/apache/storm/generated/Credentials.java |   458 -
 .../storm/generated/DRPCExceptionType.java      |    68 -
 .../storm/generated/DRPCExecutionException.java |   528 -
 .../org/apache/storm/generated/DRPCRequest.java |   507 -
 .../apache/storm/generated/DebugOptions.java    |   506 -
 .../apache/storm/generated/DistributedRPC.java  |  1328 -
 .../generated/DistributedRPCInvocations.java    |  3904 --
 .../org/apache/storm/generated/ErrorInfo.java   |   714 -
 .../storm/generated/ExecutorAggregateStats.java |   526 -
 .../apache/storm/generated/ExecutorInfo.java    |   499 -
 .../storm/generated/ExecutorSpecificStats.java  |   387 -
 .../apache/storm/generated/ExecutorStats.java   |   915 -
 .../apache/storm/generated/ExecutorSummary.java |   922 -
 .../apache/storm/generated/GetInfoOptions.java  |   422 -
 .../apache/storm/generated/GlobalStreamId.java  |   507 -
 .../org/apache/storm/generated/Grouping.java    |   800 -
 .../generated/HBAuthorizationException.java     |   406 -
 .../storm/generated/HBExecutionException.java   |   406 -
 .../org/apache/storm/generated/HBMessage.java   |   636 -
 .../apache/storm/generated/HBMessageData.java   |   640 -
 .../jvm/org/apache/storm/generated/HBNodes.java |   461 -
 .../jvm/org/apache/storm/generated/HBPulse.java |   522 -
 .../org/apache/storm/generated/HBRecords.java   |   466 -
 .../storm/generated/HBServerMessageType.java    |   113 -
 .../generated/InvalidTopologyException.java     |   406 -
 .../org/apache/storm/generated/JavaObject.java  |   561 -
 .../apache/storm/generated/JavaObjectArg.java   |   631 -
 .../generated/KeyAlreadyExistsException.java    |   406 -
 .../storm/generated/KeyNotFoundException.java   |   406 -
 .../org/apache/storm/generated/KillOptions.java |   407 -
 .../storm/generated/LSApprovedWorkers.java      |   458 -
 .../generated/LSSupervisorAssignments.java      |   471 -
 .../apache/storm/generated/LSSupervisorId.java  |   406 -
 .../apache/storm/generated/LSTopoHistory.java   |   805 -
 .../storm/generated/LSTopoHistoryList.java      |   460 -
 .../storm/generated/LSWorkerHeartbeat.java      |   755 -
 .../apache/storm/generated/ListBlobsResult.java |   556 -
 .../apache/storm/generated/LocalAssignment.java |   676 -
 .../apache/storm/generated/LocalStateData.java  |   471 -
 .../org/apache/storm/generated/LogConfig.java   |   475 -
 .../org/apache/storm/generated/LogLevel.java    |   836 -
 .../apache/storm/generated/LogLevelAction.java  |    65 -
 .../jvm/org/apache/storm/generated/Nimbus.java  | 47241 -----------------
 .../apache/storm/generated/NimbusSummary.java   |   796 -
 .../org/apache/storm/generated/NodeInfo.java    |   556 -
 .../storm/generated/NotAliveException.java      |   406 -
 .../org/apache/storm/generated/NullStruct.java  |   300 -
 .../apache/storm/generated/NumErrorsChoice.java |    65 -
 .../apache/storm/generated/ProfileAction.java   |    74 -
 .../apache/storm/generated/ProfileRequest.java  |   631 -
 .../storm/generated/ReadableBlobMeta.java       |   510 -
 .../storm/generated/RebalanceOptions.java       |   664 -
 .../storm/generated/SettableBlobMeta.java       |   567 -
 .../apache/storm/generated/ShellComponent.java  |   516 -
 .../storm/generated/SpecificAggregateStats.java |   387 -
 .../storm/generated/SpoutAggregateStats.java    |   407 -
 .../org/apache/storm/generated/SpoutSpec.java   |   514 -
 .../org/apache/storm/generated/SpoutStats.java  |   917 -
 .../apache/storm/generated/StateSpoutSpec.java  |   514 -
 .../org/apache/storm/generated/StormBase.java   |  1382 -
 .../apache/storm/generated/StormTopology.java   |  1272 -
 .../org/apache/storm/generated/StreamInfo.java  |   554 -
 .../apache/storm/generated/SubmitOptions.java   |   533 -
 .../apache/storm/generated/SupervisorInfo.java  |  1446 -
 .../storm/generated/SupervisorPageInfo.java     |   624 -
 .../storm/generated/SupervisorSummary.java      |  1265 -
 .../storm/generated/ThriftSerializedObject.java |   516 -
 .../storm/generated/TopologyActionOptions.java  |   387 -
 .../storm/generated/TopologyHistoryInfo.java    |   461 -
 .../apache/storm/generated/TopologyInfo.java    |  2144 -
 .../storm/generated/TopologyInitialStatus.java  |    62 -
 .../storm/generated/TopologyPageInfo.java       |  2757 -
 .../apache/storm/generated/TopologyStats.java   |  1094 -
 .../apache/storm/generated/TopologyStatus.java  |    68 -
 .../apache/storm/generated/TopologySummary.java |  1901 -
 .../apache/storm/generated/WorkerResources.java |   605 -
 .../apache/storm/generated/WorkerSummary.java   |  1880 -
 .../storm/grouping/CustomStreamGrouping.java    |    43 -
 .../src/jvm/org/apache/storm/grouping/Load.java |    77 -
 .../grouping/LoadAwareCustomStreamGrouping.java |    24 -
 .../grouping/LoadAwareShuffleGrouping.java      |    76 -
 .../org/apache/storm/grouping/LoadMapping.java  |    64 -
 .../storm/grouping/PartialKeyGrouping.java      |   106 -
 .../apache/storm/grouping/ShuffleGrouping.java  |    63 -
 .../org/apache/storm/hooks/BaseTaskHook.java    |    61 -
 .../org/apache/storm/hooks/BaseWorkerHook.java  |    51 -
 .../jvm/org/apache/storm/hooks/ITaskHook.java   |    38 -
 .../jvm/org/apache/storm/hooks/IWorkerHook.java |    44 -
 .../storm/hooks/SubmitterHookException.java     |    41 -
 .../apache/storm/hooks/info/BoltAckInfo.java    |    40 -
 .../storm/hooks/info/BoltExecuteInfo.java       |    40 -
 .../apache/storm/hooks/info/BoltFailInfo.java   |    40 -
 .../org/apache/storm/hooks/info/EmitInfo.java   |    44 -
 .../apache/storm/hooks/info/SpoutAckInfo.java   |    39 -
 .../apache/storm/hooks/info/SpoutFailInfo.java  |    39 -
 .../apache/storm/localizer/AsyncLocalizer.java  |   436 -
 .../org/apache/storm/localizer/ILocalizer.java  |    70 -
 .../localizer/LocalDownloadedResource.java      |   146 -
 .../apache/storm/localizer/LocalResource.java   |    44 -
 .../storm/localizer/LocalizedResource.java      |   130 -
 .../LocalizedResourceRetentionSet.java          |   140 -
 .../storm/localizer/LocalizedResourceSet.java   |   101 -
 .../org/apache/storm/localizer/Localizer.java   |   702 -
 .../storm/logging/ThriftAccessLogger.java       |    56 -
 .../logging/filters/AccessLoggingFilter.java    |    52 -
 .../storm/messaging/ConnectionWithStatus.java   |    49 -
 .../DeserializingConnectionCallback.java        |    60 -
 .../org/apache/storm/messaging/IConnection.java |    63 -
 .../storm/messaging/IConnectionCallback.java    |    31 -
 .../org/apache/storm/messaging/IContext.java    |    59 -
 .../org/apache/storm/messaging/TaskMessage.java |    53 -
 .../storm/messaging/TransportFactory.java       |    57 -
 .../apache/storm/messaging/local/Context.java   |   218 -
 .../apache/storm/messaging/netty/Client.java    |   612 -
 .../apache/storm/messaging/netty/Context.java   |   113 -
 .../storm/messaging/netty/ControlMessage.java   |    75 -
 .../messaging/netty/INettySerializable.java     |    26 -
 .../storm/messaging/netty/ISaslClient.java      |    28 -
 .../storm/messaging/netty/ISaslServer.java      |    26 -
 .../apache/storm/messaging/netty/IServer.java   |    26 -
 .../netty/KerberosSaslClientHandler.java        |   154 -
 .../netty/KerberosSaslNettyClient.java          |   202 -
 .../netty/KerberosSaslNettyClientState.java     |    31 -
 .../netty/KerberosSaslNettyServer.java          |   209 -
 .../netty/KerberosSaslNettyServerState.java     |    30 -
 .../netty/KerberosSaslServerHandler.java        |   133 -
 .../org/apache/storm/messaging/netty/Login.java |   411 -
 .../storm/messaging/netty/MessageBatch.java     |   118 -
 .../storm/messaging/netty/MessageBuffer.java    |    58 -
 .../storm/messaging/netty/MessageDecoder.java   |   144 -
 .../storm/messaging/netty/MessageEncoder.java   |    43 -
 .../netty/NettyRenameThreadFactory.java         |    56 -
 .../netty/NettyUncaughtExceptionHandler.java    |    35 -
 .../storm/messaging/netty/SaslMessageToken.java |   114 -
 .../storm/messaging/netty/SaslNettyClient.java  |   154 -
 .../messaging/netty/SaslNettyClientState.java   |    31 -
 .../storm/messaging/netty/SaslNettyServer.java  |   157 -
 .../messaging/netty/SaslNettyServerState.java   |    30 -
 .../messaging/netty/SaslStormClientHandler.java |   158 -
 .../netty/SaslStormServerAuthorizeHandler.java  |    83 -
 .../messaging/netty/SaslStormServerHandler.java |   153 -
 .../apache/storm/messaging/netty/SaslUtils.java |    68 -
 .../apache/storm/messaging/netty/Server.java    |   273 -
 .../messaging/netty/StormClientHandler.java     |    90 -
 .../netty/StormClientPipelineFactory.java       |    56 -
 .../messaging/netty/StormServerHandler.java     |    74 -
 .../netty/StormServerPipelineFactory.java       |    57 -
 .../metric/ClusterMetricsConsumerExecutor.java  |    89 -
 .../apache/storm/metric/EventLoggerBolt.java    |    67 -
 .../storm/metric/FileBasedEventLogger.java      |   122 -
 .../metric/HttpForwardingMetricsConsumer.java   |    85 -
 .../metric/HttpForwardingMetricsServer.java     |   118 -
 .../org/apache/storm/metric/IEventLogger.java   |    70 -
 .../metric/LoggingClusterMetricsConsumer.java   |    86 -
 .../storm/metric/LoggingMetricsConsumer.java    |    76 -
 .../storm/metric/MetricsConsumerBolt.java       |   146 -
 .../storm/metric/StormMetricsRegistry.java      |    88 -
 .../jvm/org/apache/storm/metric/SystemBolt.java |   160 -
 .../storm/metric/api/AssignableMetric.java      |    34 -
 .../apache/storm/metric/api/CombinedMetric.java |    38 -
 .../apache/storm/metric/api/CountMetric.java    |    39 -
 .../org/apache/storm/metric/api/DataPoint.java  |    43 -
 .../metric/api/IClusterMetricsConsumer.java     |    63 -
 .../org/apache/storm/metric/api/ICombiner.java  |    23 -
 .../org/apache/storm/metric/api/IMetric.java    |    30 -
 .../storm/metric/api/IMetricsConsumer.java      |    81 -
 .../org/apache/storm/metric/api/IReducer.java   |    24 -
 .../storm/metric/api/IStatefulObject.java       |    22 -
 .../apache/storm/metric/api/MeanReducer.java    |    53 -
 .../storm/metric/api/MultiCountMetric.java      |    45 -
 .../storm/metric/api/MultiReducedMetric.java    |    50 -
 .../apache/storm/metric/api/ReducedMetric.java  |    38 -
 .../apache/storm/metric/api/StateMetric.java    |    31 -
 .../metric/api/rpc/AssignableShellMetric.java   |    30 -
 .../metric/api/rpc/CombinedShellMetric.java     |    31 -
 .../storm/metric/api/rpc/CountShellMetric.java  |    37 -
 .../storm/metric/api/rpc/IShellMetric.java      |    31 -
 .../metric/api/rpc/ReducedShellMetric.java      |    32 -
 .../apache/storm/metric/cgroup/CGroupCpu.java   |    70 -
 .../storm/metric/cgroup/CGroupCpuGuarantee.java |    52 -
 .../storm/metric/cgroup/CGroupMemoryLimit.java  |    39 -
 .../storm/metric/cgroup/CGroupMemoryUsage.java  |    39 -
 .../storm/metric/cgroup/CGroupMetricsBase.java  |   114 -
 .../storm/metric/filter/FilterByMetricName.java |   110 -
 .../storm/metric/filter/MetricsFilter.java      |    26 -
 .../metric/internal/CountStatAndMetric.java     |   211 -
 .../metric/internal/LatencyStatAndMetric.java   |   262 -
 .../storm/metric/internal/MetricStatTimer.java  |    27 -
 .../internal/MultiCountStatAndMetric.java       |   112 -
 .../internal/MultiLatencyStatAndMetric.java     |   109 -
 .../storm/metric/internal/RateTracker.java      |   139 -
 .../storm/metric/util/DataPointExpander.java    |    79 -
 .../jvm/org/apache/storm/multilang/BoltMsg.java |    79 -
 .../org/apache/storm/multilang/ISerializer.java |    82 -
 .../apache/storm/multilang/JsonSerializer.java  |   203 -
 .../storm/multilang/NoOutputException.java      |    40 -
 .../org/apache/storm/multilang/ShellMsg.java    |   184 -
 .../org/apache/storm/multilang/SpoutMsg.java    |    50 -
 .../AbstractDNSToSwitchMapping.java             |    96 -
 .../networktopography/DNSToSwitchMapping.java   |    49 -
 .../DefaultRackDNSToSwitchMapping.java          |    52 -
 .../storm/nimbus/DefaultTopologyValidator.java  |    31 -
 .../org/apache/storm/nimbus/ILeaderElector.java |    73 -
 .../nimbus/ITopologyActionNotifierPlugin.java   |    43 -
 .../apache/storm/nimbus/ITopologyValidator.java |    28 -
 .../jvm/org/apache/storm/nimbus/NimbusInfo.java |   120 -
 .../storm/pacemaker/IServerMessageHandler.java  |    25 -
 .../org/apache/storm/pacemaker/Pacemaker.java   |   218 -
 .../apache/storm/pacemaker/PacemakerClient.java |   274 -
 .../storm/pacemaker/PacemakerClientHandler.java |    80 -
 .../storm/pacemaker/PacemakerClientPool.java    |   113 -
 .../pacemaker/PacemakerConnectionException.java |    24 -
 .../apache/storm/pacemaker/PacemakerServer.java |   168 -
 .../storm/pacemaker/codec/ThriftDecoder.java    |    69 -
 .../storm/pacemaker/codec/ThriftEncoder.java    |   110 -
 .../pacemaker/codec/ThriftNettyClientCodec.java |    97 -
 .../pacemaker/codec/ThriftNettyServerCodec.java |    99 -
 .../jvm/org/apache/storm/scheduler/Cluster.java |   855 -
 .../storm/scheduler/DefaultScheduler.java       |   111 -
 .../apache/storm/scheduler/EvenScheduler.java   |   171 -
 .../apache/storm/scheduler/ExecutorDetails.java |    54 -
 .../jvm/org/apache/storm/scheduler/INimbus.java |    49 -
 .../org/apache/storm/scheduler/IScheduler.java  |    40 -
 .../org/apache/storm/scheduler/ISupervisor.java |    45 -
 .../storm/scheduler/IsolationScheduler.java     |   425 -
 .../storm/scheduler/SchedulerAssignment.java    |    61 -
 .../scheduler/SchedulerAssignmentImpl.java      |   135 -
 .../storm/scheduler/SupervisorDetails.java      |   138 -
 .../org/apache/storm/scheduler/Topologies.java  |    87 -
 .../apache/storm/scheduler/TopologyDetails.java |   516 -
 .../org/apache/storm/scheduler/WorkerSlot.java  |    81 -
 .../scheduler/multitenant/DefaultPool.java      |   219 -
 .../storm/scheduler/multitenant/FreePool.java   |   125 -
 .../scheduler/multitenant/IsolatedPool.java     |   371 -
 .../multitenant/MultitenantScheduler.java       |    98 -
 .../storm/scheduler/multitenant/Node.java       |   340 -
 .../storm/scheduler/multitenant/NodePool.java   |   296 -
 .../storm/scheduler/resource/Component.java     |    54 -
 .../storm/scheduler/resource/RAS_Node.java      |   529 -
 .../storm/scheduler/resource/RAS_Nodes.java     |   138 -
 .../resource/ResourceAwareScheduler.java        |   435 -
 .../storm/scheduler/resource/ResourceUtils.java |   206 -
 .../scheduler/resource/SchedulingResult.java    |   116 -
 .../scheduler/resource/SchedulingState.java     |    56 -
 .../scheduler/resource/SchedulingStatus.java    |    40 -
 .../apache/storm/scheduler/resource/User.java   |   350 -
 .../eviction/DefaultEvictionStrategy.java       |   126 -
 .../strategies/eviction/IEvictionStrategy.java  |    42 -
 .../DefaultSchedulingPriorityStrategy.java      |    80 -
 .../priority/ISchedulingPriorityStrategy.java   |    36 -
 .../DefaultResourceAwareStrategy.java           |   757 -
 .../strategies/scheduling/IStrategy.java        |    47 -
 .../storm/security/INimbusCredentialPlugin.java |    47 -
 .../auth/AbstractSaslClientCallbackHandler.java |    76 -
 .../auth/AbstractSaslServerCallbackHandler.java |    94 -
 .../apache/storm/security/auth/AuthUtils.java   |   419 -
 .../org/apache/storm/security/auth/AutoSSL.java |   161 -
 .../auth/DefaultHttpCredentialsPlugin.java      |    96 -
 .../security/auth/DefaultPrincipalToLocal.java  |    42 -
 .../apache/storm/security/auth/IAuthorizer.java |    53 -
 .../storm/security/auth/IAutoCredentials.java   |    55 -
 .../security/auth/ICredentialsRenewer.java      |    40 -
 .../auth/IGroupMappingServiceProvider.java      |    42 -
 .../security/auth/IHttpCredentialsPlugin.java   |    48 -
 .../storm/security/auth/IPrincipalToLocal.java  |    41 -
 .../storm/security/auth/ITransportPlugin.java   |    57 -
 .../security/auth/KerberosPrincipalToLocal.java |    45 -
 .../storm/security/auth/NimbusPrincipal.java    |    29 -
 .../apache/storm/security/auth/ReqContext.java  |   152 -
 .../security/auth/SaslTransportPlugin.java      |   180 -
 .../security/auth/ShellBasedGroupsMapping.java  |    99 -
 .../security/auth/SimpleTransportPlugin.java    |   163 -
 .../security/auth/SingleUserPrincipal.java      |    53 -
 .../storm/security/auth/TBackoffConnect.java    |    82 -
 .../storm/security/auth/ThriftClient.java       |   124 -
 .../security/auth/ThriftConnectionType.java     |    86 -
 .../storm/security/auth/ThriftServer.java       |    78 -
 .../auth/authorizer/DRPCAuthorizerBase.java     |    63 -
 .../authorizer/DRPCSimpleACLAuthorizer.java     |   177 -
 .../auth/authorizer/DenyAuthorizer.java         |    47 -
 .../authorizer/ImpersonationAuthorizer.java     |   172 -
 .../auth/authorizer/NoopAuthorizer.java         |    47 -
 .../auth/authorizer/SimpleACLAuthorizer.java    |   173 -
 .../authorizer/SimpleWhitelistAuthorizer.java   |    60 -
 .../auth/digest/ClientCallbackHandler.java      |    58 -
 .../auth/digest/DigestSaslTransportPlugin.java  |    68 -
 .../auth/digest/ServerCallbackHandler.java      |    85 -
 .../storm/security/auth/kerberos/AutoTGT.java   |   269 -
 .../auth/kerberos/AutoTGTKrb5LoginModule.java   |   112 -
 .../kerberos/AutoTGTKrb5LoginModuleTest.java    |    44 -
 .../auth/kerberos/ClientCallbackHandler.java    |   104 -
 .../kerberos/KerberosSaslTransportPlugin.java   |   262 -
 .../security/auth/kerberos/NoOpTTrasport.java   |    54 -
 .../auth/kerberos/ServerCallbackHandler.java    |    96 -
 .../auth/kerberos/jaas_kerberos_cluster.conf    |    49 -
 .../auth/kerberos/jaas_kerberos_launcher.conf   |    31 -
 .../auth/plain/PlainClientCallbackHandler.java  |    31 -
 .../auth/plain/PlainSaslTransportPlugin.java    |    71 -
 .../auth/plain/PlainServerCallbackHandler.java  |    55 -
 .../security/auth/plain/SaslPlainServer.java    |   158 -
 .../serialization/BlowfishTupleSerializer.java  |    92 -
 .../storm/serialization/DefaultKryoFactory.java |    64 -
 .../DefaultSerializationDelegate.java           |    58 -
 .../GzipBridgeSerializationDelegate.java        |    65 -
 .../GzipBridgeThriftSerializationDelegate.java  |    64 -
 .../GzipSerializationDelegate.java              |    64 -
 .../GzipThriftSerializationDelegate.java        |    56 -
 .../storm/serialization/IKryoDecorator.java     |    23 -
 .../storm/serialization/IKryoFactory.java       |    40 -
 .../storm/serialization/ITupleDeserializer.java |    24 -
 .../storm/serialization/ITupleSerializer.java   |    26 -
 .../serialization/KryoTupleDeserializer.java    |    56 -
 .../serialization/KryoTupleSerializer.java      |    60 -
 .../serialization/KryoValuesDeserializer.java   |    50 -
 .../serialization/KryoValuesSerializer.java     |    58 -
 .../serialization/SerializableSerializer.java   |    61 -
 .../serialization/SerializationDelegate.java    |    35 -
 .../serialization/SerializationFactory.java     |   240 -
 .../ThriftSerializationDelegate.java            |    52 -
 .../types/ArrayListSerializer.java              |    32 -
 .../serialization/types/HashMapSerializer.java  |    32 -
 .../serialization/types/HashSetSerializer.java  |    32 -
 .../types/ListDelegateSerializer.java           |    32 -
 .../org/apache/storm/spout/CheckPointState.java |   172 -
 .../org/apache/storm/spout/CheckpointSpout.java |   232 -
 .../storm/spout/IMultiSchemableSpout.java       |    23 -
 .../org/apache/storm/spout/ISchemableSpout.java |    24 -
 .../src/jvm/org/apache/storm/spout/ISpout.java  |   105 -
 .../storm/spout/ISpoutOutputCollector.java      |    32 -
 .../apache/storm/spout/ISpoutWaitStrategy.java  |    34 -
 .../jvm/org/apache/storm/spout/MultiScheme.java |    29 -
 .../storm/spout/NothingEmptyEmitStrategy.java   |    31 -
 .../org/apache/storm/spout/RawMultiScheme.java  |    40 -
 .../jvm/org/apache/storm/spout/RawScheme.java   |    37 -
 .../src/jvm/org/apache/storm/spout/Scheme.java  |    29 -
 .../apache/storm/spout/SchemeAsMultiScheme.java |    42 -
 .../jvm/org/apache/storm/spout/ShellSpout.java  |   327 -
 .../storm/spout/SleepSpoutWaitStrategy.java     |    41 -
 .../storm/spout/SpoutOutputCollector.java       |   143 -
 .../storm/state/DefaultStateSerializer.java     |    77 -
 .../jvm/org/apache/storm/state/IStateSpout.java |    29 -
 .../storm/state/IStateSpoutOutputCollector.java |    22 -
 .../apache/storm/state/ISubscribedState.java    |    25 -
 .../state/ISynchronizeOutputCollector.java      |    24 -
 .../storm/state/InMemoryKeyValueState.java      |   126 -
 .../state/InMemoryKeyValueStateProvider.java    |    43 -
 .../org/apache/storm/state/KeyValueState.java   |    58 -
 .../jvm/org/apache/storm/state/Serializer.java  |    30 -
 .../src/jvm/org/apache/storm/state/State.java   |    55 -
 .../org/apache/storm/state/StateFactory.java    |    69 -
 .../org/apache/storm/state/StateProvider.java   |    38 -
 .../storm/state/StateSpoutOutputCollector.java  |    28 -
 .../storm/state/SynchronizeOutputCollector.java |    30 -
 .../apache/storm/stats/BoltExecutorStats.java   |   106 -
 .../jvm/org/apache/storm/stats/CommonStats.java |   114 -
 .../apache/storm/stats/SpoutExecutorStats.java  |    76 -
 .../jvm/org/apache/storm/stats/StatsUtil.java   |  2612 -
 .../src/jvm/org/apache/storm/streams/Edge.java  |    41 -
 .../org/apache/storm/streams/GroupingInfo.java  |   100 -
 .../src/jvm/org/apache/storm/streams/Node.java  |   170 -
 .../src/jvm/org/apache/storm/streams/Pair.java  |   103 -
 .../org/apache/storm/streams/PairStream.java    |   525 -
 .../org/apache/storm/streams/PartitionNode.java |    37 -
 .../org/apache/storm/streams/ProcessorBolt.java |    70 -
 .../storm/streams/ProcessorBoltDelegate.java    |   348 -
 .../org/apache/storm/streams/ProcessorNode.java |    83 -
 .../apache/storm/streams/RefCountedTuple.java   |    64 -
 .../jvm/org/apache/storm/streams/SinkNode.java  |    44 -
 .../jvm/org/apache/storm/streams/SpoutNode.java |    48 -
 .../storm/streams/StatefulProcessorBolt.java    |   116 -
 .../jvm/org/apache/storm/streams/Stream.java    |   515 -
 .../org/apache/storm/streams/StreamBolt.java    |    38 -
 .../org/apache/storm/streams/StreamBuilder.java |   602 -
 .../org/apache/storm/streams/StreamState.java   |    43 -
 .../org/apache/storm/streams/StreamUtil.java    |    69 -
 .../storm/streams/StreamsEdgeFactory.java       |    29 -
 .../org/apache/storm/streams/UniqueIdGen.java   |    56 -
 .../org/apache/storm/streams/WindowNode.java    |    39 -
 .../storm/streams/WindowedProcessorBolt.java    |   137 -
 .../storm/streams/operations/BiFunction.java    |    37 -
 .../streams/operations/CombinerAggregator.java  |    97 -
 .../storm/streams/operations/Consumer.java      |    32 -
 .../streams/operations/FlatMapFunction.java     |    27 -
 .../storm/streams/operations/Function.java      |    34 -
 .../streams/operations/IdentityFunction.java    |    31 -
 .../storm/streams/operations/Operation.java     |    26 -
 .../streams/operations/PairFlatMapFunction.java |    30 -
 .../storm/streams/operations/PairFunction.java  |    30 -
 .../streams/operations/PairValueJoiner.java     |    40 -
 .../storm/streams/operations/Predicate.java     |    33 -
 .../storm/streams/operations/PrintConsumer.java |    30 -
 .../storm/streams/operations/Reducer.java       |    35 -
 .../storm/streams/operations/StateUpdater.java  |    67 -
 .../storm/streams/operations/ValueJoiner.java   |    36 -
 .../streams/operations/aggregators/Count.java   |    47 -
 .../streams/operations/aggregators/LongSum.java |    45 -
 .../operations/mappers/PairValueMapper.java     |    51 -
 .../operations/mappers/TupleValueMapper.java    |    30 -
 .../operations/mappers/TupleValueMappers.java   |   174 -
 .../streams/operations/mappers/ValueMapper.java |    45 -
 .../operations/mappers/ValuesMapper.java        |    48 -
 .../processors/AggregateByKeyProcessor.java     |    78 -
 .../streams/processors/AggregateProcessor.java  |    69 -
 .../storm/streams/processors/BaseProcessor.java |   109 -
 .../streams/processors/BatchProcessor.java      |    25 -
 .../streams/processors/BranchProcessor.java     |    41 -
 .../processors/ChainedProcessorContext.java     |    66 -
 .../processors/EmittingProcessorContext.java    |   155 -
 .../streams/processors/FilterProcessor.java     |    35 -
 .../streams/processors/FlatMapProcessor.java    |    35 -
 .../processors/FlatMapValuesProcessor.java      |    36 -
 .../streams/processors/ForEachProcessor.java    |    33 -
 .../processors/ForwardingProcessorContext.java  |   102 -
 .../storm/streams/processors/JoinProcessor.java |   152 -
 .../storm/streams/processors/MapProcessor.java  |    33 -
 .../streams/processors/MapValuesProcessor.java  |    34 -
 .../MergeAggregateByKeyProcessor.java           |    54 -
 .../processors/MergeAggregateProcessor.java     |    47 -
 .../storm/streams/processors/PeekProcessor.java |    34 -
 .../storm/streams/processors/Processor.java     |    54 -
 .../streams/processors/ProcessorContext.java    |    62 -
 .../processors/ReduceByKeyProcessor.java        |    52 -
 .../streams/processors/ReduceProcessor.java     |    43 -
 .../streams/processors/StateQueryProcessor.java |    48 -
 .../streams/processors/StatefulProcessor.java   |    36 -
 .../processors/UpdateStateByKeyProcessor.java   |    49 -
 .../org/apache/storm/streams/tuple/Tuple10.java |   112 -
 .../org/apache/storm/streams/tuple/Tuple3.java  |    70 -
 .../org/apache/storm/streams/tuple/Tuple4.java  |    76 -
 .../org/apache/storm/streams/tuple/Tuple5.java  |    82 -
 .../org/apache/storm/streams/tuple/Tuple6.java  |    89 -
 .../org/apache/storm/streams/tuple/Tuple7.java  |    94 -
 .../org/apache/storm/streams/tuple/Tuple8.java  |   100 -
 .../org/apache/storm/streams/tuple/Tuple9.java  |   106 -
 .../storm/streams/windowing/BaseWindow.java     |    64 -
 .../storm/streams/windowing/SlidingWindows.java |   151 -
 .../streams/windowing/TumblingWindows.java      |   119 -
 .../apache/storm/streams/windowing/Window.java  |    70 -
 .../storm/task/GeneralTopologyContext.java      |   202 -
 .../src/jvm/org/apache/storm/task/IBolt.java    |    84 -
 .../org/apache/storm/task/IErrorReporter.java   |    22 -
 .../org/apache/storm/task/IMetricsContext.java  |    31 -
 .../org/apache/storm/task/IOutputCollector.java |    33 -
 .../org/apache/storm/task/OutputCollector.java  |   236 -
 .../jvm/org/apache/storm/task/ShellBolt.java    |   435 -
 .../org/apache/storm/task/TopologyContext.java  |   402 -
 .../storm/task/WorkerTopologyContext.java       |   106 -
 .../apache/storm/testing/AckFailDelegate.java   |    25 -
 .../apache/storm/testing/AckFailMapTracker.java |    52 -
 .../org/apache/storm/testing/AckTracker.java    |    52 -
 .../AlternateRackDNSToSwitchMapping.java        |    65 -
 .../apache/storm/testing/BatchNumberList.java   |    72 -
 .../apache/storm/testing/BatchProcessWord.java  |    39 -
 .../org/apache/storm/testing/BatchRepeatA.java  |    48 -
 .../org/apache/storm/testing/BoltTracker.java   |    42 -
 .../apache/storm/testing/CompletableSpout.java  |    39 -
 .../storm/testing/CompleteTopologyParam.java    |   103 -
 .../apache/storm/testing/CountingBatchBolt.java |    55 -
 .../storm/testing/CountingCommitBolt.java       |    57 -
 .../org/apache/storm/testing/FeederSpout.java   |   107 -
 .../org/apache/storm/testing/FixedTuple.java    |    42 -
 .../apache/storm/testing/FixedTupleSpout.java   |   184 -
 .../testing/ForwardingMetricsConsumer.java      |   100 -
 .../org/apache/storm/testing/IdentityBolt.java  |    42 -
 .../storm/testing/InProcessZookeeper.java       |    59 -
 .../apache/storm/testing/IntegrationTest.java   |    38 -
 .../storm/testing/KeyedCountingBatchBolt.java   |    61 -
 .../testing/KeyedCountingCommitterBolt.java     |    24 -
 .../storm/testing/KeyedSummingBatchBolt.java    |    61 -
 .../storm/testing/MemoryTransactionalSpout.java |   180 -
 .../testing/MemoryTransactionalSpoutMeta.java   |    38 -
 .../apache/storm/testing/MkClusterParam.java    |    69 -
 .../org/apache/storm/testing/MkTupleParam.java  |    58 -
 .../org/apache/storm/testing/MockedSources.java |    68 -
 .../jvm/org/apache/storm/testing/NGrouping.java |    48 -
 .../storm/testing/NonRichBoltTracker.java       |    51 -
 .../testing/OpaqueMemoryTransactionalSpout.java |   186 -
 .../apache/storm/testing/PrepareBatchBolt.java  |    52 -
 .../storm/testing/PythonShellMetricsBolt.java   |    53 -
 .../storm/testing/PythonShellMetricsSpout.java  |    56 -
 .../testing/SingleUserSimpleTransport.java      |    37 -
 .../org/apache/storm/testing/SpoutTracker.java  |   111 -
 .../storm/testing/TestAggregatesCounter.java    |    63 -
 .../org/apache/storm/testing/TestConfBolt.java  |    62 -
 .../apache/storm/testing/TestEventLogSpout.java |   144 -
 .../storm/testing/TestEventOrderCheckBolt.java  |    76 -
 .../apache/storm/testing/TestGlobalCount.java   |    60 -
 .../jvm/org/apache/storm/testing/TestJob.java   |    43 -
 .../apache/storm/testing/TestKryoDecorator.java |    31 -
 .../apache/storm/testing/TestPlannerBolt.java   |    45 -
 .../apache/storm/testing/TestPlannerSpout.java  |    85 -
 .../org/apache/storm/testing/TestSerObject.java |    37 -
 .../storm/testing/TestWordBytesCounter.java     |    27 -
 .../apache/storm/testing/TestWordCounter.java   |    65 -
 .../org/apache/storm/testing/TestWordSpout.java |    86 -
 .../jvm/org/apache/storm/testing/TmpPath.java   |    69 -
 .../apache/storm/testing/TrackedTopology.java   |   135 -
 .../apache/storm/testing/TupleCaptureBolt.java  |    83 -
 .../staticmocking/MockedClientZookeeper.java    |    32 +
 .../topology/BaseConfigurationDeclarer.java     |    84 -
 .../topology/BaseStatefulBoltExecutor.java      |   209 -
 .../storm/topology/BasicBoltExecutor.java       |    67 -
 .../storm/topology/BasicOutputCollector.java    |    72 -
 .../org/apache/storm/topology/BoltDeclarer.java |    26 -
 .../topology/CheckpointTupleForwarder.java      |   103 -
 .../ComponentConfigurationDeclarer.java         |    29 -
 .../storm/topology/ConfigurableTopology.java    |   184 -
 .../apache/storm/topology/FailedException.java  |    36 -
 .../org/apache/storm/topology/IBasicBolt.java   |    33 -
 .../storm/topology/IBasicOutputCollector.java   |    29 -
 .../org/apache/storm/topology/IComponent.java   |    44 -
 .../org/apache/storm/topology/IRichBolt.java    |    29 -
 .../org/apache/storm/topology/IRichSpout.java   |    29 -
 .../apache/storm/topology/IRichStateSpout.java  |    25 -
 .../apache/storm/topology/IStatefulBolt.java    |    48 -
 .../storm/topology/IStatefulComponent.java      |    56 -
 .../storm/topology/IStatefulWindowedBolt.java   |    26 -
 .../apache/storm/topology/IWindowedBolt.java    |    51 -
 .../apache/storm/topology/InputDeclarer.java    |   184 -
 .../storm/topology/OutputFieldsDeclarer.java    |    32 -
 .../storm/topology/OutputFieldsGetter.java      |    53 -
 .../storm/topology/ReportedFailedException.java |    36 -
 .../apache/storm/topology/ResourceDeclarer.java |    28 -
 .../apache/storm/topology/SpoutDeclarer.java    |    22 -
 .../storm/topology/StatefulBoltExecutor.java    |   193 -
 .../topology/StatefulWindowedBoltExecutor.java  |   372 -
 .../apache/storm/topology/TopologyBuilder.java  |   588 -
 .../topology/TupleFieldTimestampExtractor.java  |    48 -
 .../storm/topology/WindowedBoltExecutor.java    |   363 -
 .../storm/topology/base/BaseBasicBolt.java      |    33 -
 .../storm/topology/base/BaseBatchBolt.java      |    24 -
 .../storm/topology/base/BaseComponent.java      |    28 -
 ...BaseOpaquePartitionedTransactionalSpout.java |    25 -
 .../base/BasePartitionedTransactionalSpout.java |    25 -
 .../storm/topology/base/BaseRichBolt.java       |    26 -
 .../storm/topology/base/BaseRichSpout.java      |    46 -
 .../storm/topology/base/BaseStatefulBolt.java   |    64 -
 .../topology/base/BaseStatefulWindowedBolt.java |   168 -
 .../base/BaseTickTupleAwareRichBolt.java        |    60 -
 .../topology/base/BaseTransactionalBolt.java    |    24 -
 .../topology/base/BaseTransactionalSpout.java   |    24 -
 .../storm/topology/base/BaseWindowedBolt.java   |   365 -
 .../apache/storm/transactional/ICommitter.java  |    26 -
 .../ICommitterTransactionalSpout.java           |    31 -
 .../transactional/ITransactionalSpout.java      |    94 -
 .../storm/transactional/TransactionAttempt.java |    61 -
 .../TransactionalSpoutBatchExecutor.java        |    96 -
 .../TransactionalSpoutCoordinator.java          |   212 -
 .../TransactionalTopologyBuilder.java           |   521 -
 .../IOpaquePartitionedTransactionalSpout.java   |    56 -
 .../IPartitionedTransactionalSpout.java         |    69 -
 ...uePartitionedTransactionalSpoutExecutor.java |   160 -
 .../PartitionedTransactionalSpoutExecutor.java  |   142 -
 .../state/RotatingTransactionalState.java       |   149 -
 .../state/TestTransactionalState.java           |    47 -
 .../transactional/state/TransactionalState.java |   173 -
 .../apache/storm/trident/JoinOutFieldsMode.java |    33 -
 .../jvm/org/apache/storm/trident/JoinType.java  |    30 -
 .../jvm/org/apache/storm/trident/Stream.java    |   897 -
 .../org/apache/storm/trident/TridentState.java  |    59 -
 .../apache/storm/trident/TridentTopology.java   |   986 -
 .../trident/drpc/ReturnResultsReducer.java      |   128 -
 .../fluent/ChainedAggregatorDeclarer.java       |   183 -
 .../fluent/ChainedFullAggregatorDeclarer.java   |    32 -
 .../ChainedPartitionAggregatorDeclarer.java     |    32 -
 .../trident/fluent/GlobalAggregationScheme.java |    26 -
 .../storm/trident/fluent/GroupedStream.java     |   174 -
 .../trident/fluent/IAggregatableStream.java     |    31 -
 .../fluent/IChainedAggregatorDeclarer.java      |    24 -
 .../storm/trident/fluent/UniqueIdGen.java       |    38 -
 .../storm/trident/graph/GraphGrouper.java       |   119 -
 .../org/apache/storm/trident/graph/Group.java   |   150 -
 .../storm/trident/operation/Aggregator.java     |    26 -
 .../storm/trident/operation/Assembly.java       |    45 -
 .../storm/trident/operation/BaseAggregator.java |    23 -
 .../storm/trident/operation/BaseFilter.java     |    23 -
 .../storm/trident/operation/BaseFunction.java   |    23 -
 .../trident/operation/BaseMultiReducer.java     |    33 -
 .../storm/trident/operation/BaseOperation.java  |    47 -
 .../trident/operation/CombinerAggregator.java   |    29 -
 .../storm/trident/operation/Consumer.java       |    35 -
 .../operation/DefaultResourceDeclarer.java      |    68 -
 .../storm/trident/operation/EachOperation.java  |    22 -
 .../apache/storm/trident/operation/Filter.java  |    48 -
 .../trident/operation/FlatMapFunction.java      |    37 -
 .../storm/trident/operation/Function.java       |    92 -
 .../trident/operation/GroupedMultiReducer.java  |    31 -
 .../trident/operation/ITridentResource.java     |    32 -
 .../storm/trident/operation/MapFunction.java    |    36 -
 .../storm/trident/operation/MultiReducer.java   |    31 -
 .../storm/trident/operation/Operation.java      |    46 -
 .../OperationAwareFlatMapFunction.java          |    24 -
 .../operation/OperationAwareMapFunction.java    |    24 -
 .../trident/operation/ReducerAggregator.java    |    26 -
 .../trident/operation/TridentCollector.java     |    52 -
 .../operation/TridentMultiReducerContext.java   |    36 -
 .../operation/TridentOperationContext.java      |    65 -
 .../operation/builtin/ComparisonAggregator.java |    91 -
 .../storm/trident/operation/builtin/Count.java  |    41 -
 .../storm/trident/operation/builtin/Debug.java  |    62 -
 .../storm/trident/operation/builtin/Equals.java |    38 -
 .../trident/operation/builtin/FilterNull.java   |    35 -
 .../storm/trident/operation/builtin/FirstN.java |   130 -
 .../storm/trident/operation/builtin/MapGet.java |    38 -
 .../storm/trident/operation/builtin/Max.java    |    37 -
 .../operation/builtin/MaxWithComparator.java    |    51 -
 .../storm/trident/operation/builtin/Min.java    |    36 -
 .../operation/builtin/MinWithComparator.java    |    51 -
 .../storm/trident/operation/builtin/Negate.java |    76 -
 .../trident/operation/builtin/SnapshotGet.java  |    44 -
 .../storm/trident/operation/builtin/Sum.java    |    42 -
 .../operation/builtin/TupleCollectionGet.java   |    46 -
 .../operation/impl/CaptureCollector.java        |    42 -
 .../operation/impl/ChainedAggregatorImpl.java   |   113 -
 .../trident/operation/impl/ChainedResult.java   |    53 -
 .../operation/impl/CombinerAggStateUpdater.java |    56 -
 .../impl/CombinerAggregatorCombineImpl.java     |    61 -
 .../impl/CombinerAggregatorInitImpl.java        |    49 -
 .../operation/impl/ConsumerExecutor.java        |    38 -
 .../trident/operation/impl/FilterExecutor.java  |    53 -
 .../operation/impl/FlatMapFunctionExecutor.java |    64 -
 .../operation/impl/GlobalBatchToPartition.java  |    29 -
 .../trident/operation/impl/GroupCollector.java  |    48 -
 .../operation/impl/GroupedAggregator.java       |    96 -
 .../impl/GroupedMultiReducerExecutor.java       |    95 -
 .../operation/impl/IdentityMultiReducer.java    |    51 -
 .../impl/IndexHashBatchToPartition.java         |    29 -
 .../storm/trident/operation/impl/JoinState.java |    39 -
 .../operation/impl/JoinerMultiReducer.java      |   142 -
 .../operation/impl/MapFunctionExecutor.java     |    62 -
 ...PreservingFieldsOrderJoinerMultiReducer.java |   175 -
 .../operation/impl/ReducerAggStateUpdater.java  |    53 -
 .../operation/impl/ReducerAggregatorImpl.java   |    56 -
 .../storm/trident/operation/impl/Result.java    |    27 -
 .../operation/impl/SingleEmitAggregator.java    |    95 -
 .../trident/operation/impl/TrueFilter.java      |    40 -
 .../storm/trident/partition/GlobalGrouping.java |    42 -
 .../trident/partition/IdentityGrouping.java     |    59 -
 .../trident/partition/IndexHashGrouping.java    |    55 -
 .../storm/trident/planner/BridgeReceiver.java   |    38 -
 .../org/apache/storm/trident/planner/Node.java  |    69 -
 .../storm/trident/planner/NodeStateInfo.java    |    31 -
 .../storm/trident/planner/PartitionNode.java    |    50 -
 .../storm/trident/planner/ProcessorContext.java |    29 -
 .../storm/trident/planner/ProcessorNode.java    |    38 -
 .../apache/storm/trident/planner/SpoutNode.java |    39 -
 .../storm/trident/planner/SubtopologyBolt.java  |   246 -
 .../storm/trident/planner/TridentProcessor.java |    40 -
 .../storm/trident/planner/TupleReceiver.java    |    27 -
 .../planner/processor/AggregateProcessor.java   |    84 -
 .../planner/processor/AppendCollector.java      |    62 -
 .../planner/processor/EachProcessor.java        |    80 -
 .../planner/processor/FreshCollector.java       |    59 -
 .../trident/planner/processor/MapProcessor.java |    87 -
 .../processor/MultiReducerProcessor.java        |    93 -
 .../processor/PartitionPersistProcessor.java    |   107 -
 .../planner/processor/ProjectedProcessor.java   |    73 -
 .../planner/processor/StateQueryProcessor.java  |   107 -
 .../planner/processor/TridentContext.java       |    76 -
 .../storm/trident/spout/BatchSpoutExecutor.java |    92 -
 .../apache/storm/trident/spout/IBatchID.java    |    24 -
 .../apache/storm/trident/spout/IBatchSpout.java |    33 -
 .../trident/spout/ICommitterTridentSpout.java   |    31 -
 .../spout/IOpaquePartitionedTridentSpout.java   |    87 -
 .../trident/spout/IPartitionedTridentSpout.java |    77 -
 .../storm/trident/spout/ISpoutPartition.java    |    25 -
 .../storm/trident/spout/ITridentDataSource.java |    26 -
 .../storm/trident/spout/ITridentSpout.java      |   125 -
 .../OpaquePartitionedTridentSpoutExecutor.java  |   234 -
 .../spout/PartitionedTridentSpoutExecutor.java  |   188 -
 .../trident/spout/RichSpoutBatchExecutor.java   |   204 -
 .../storm/trident/spout/RichSpoutBatchId.java   |    49 -
 .../spout/RichSpoutBatchIdSerializer.java       |    38 -
 .../trident/spout/RichSpoutBatchTriggerer.java  |   182 -
 .../trident/spout/TridentSpoutCoordinator.java  |    94 -
 .../trident/spout/TridentSpoutExecutor.java     |   138 -
 .../storm/trident/state/BaseQueryFunction.java  |    25 -
 .../storm/trident/state/BaseStateUpdater.java   |    25 -
 .../trident/state/CombinerValueUpdater.java     |    45 -
 .../storm/trident/state/ITupleCollection.java   |    26 -
 .../state/JSONNonTransactionalSerializer.java   |    45 -
 .../trident/state/JSONOpaqueSerializer.java     |    52 -
 .../state/JSONTransactionalSerializer.java      |    50 -
 .../apache/storm/trident/state/OpaqueValue.java |    75 -
 .../storm/trident/state/QueryFunction.java      |    28 -
 .../storm/trident/state/ReadOnlyState.java      |    31 -
 .../trident/state/ReducerValueUpdater.java      |    41 -
 .../apache/storm/trident/state/Serializer.java  |    26 -
 .../org/apache/storm/trident/state/State.java   |    39 -
 .../storm/trident/state/StateFactory.java       |    26 -
 .../apache/storm/trident/state/StateSpec.java   |    30 -
 .../apache/storm/trident/state/StateType.java   |    25 -
 .../storm/trident/state/StateUpdater.java       |    33 -
 .../storm/trident/state/TransactionalValue.java |    44 -
 .../storm/trident/state/ValueUpdater.java       |    23 -
 .../trident/state/map/CachedBatchReadsMap.java  |    80 -
 .../storm/trident/state/map/CachedMap.java      |    78 -
 .../storm/trident/state/map/IBackingMap.java    |    26 -
 .../state/map/MapCombinerAggStateUpdater.java   |    84 -
 .../state/map/MapReducerAggStateUpdater.java    |    91 -
 .../storm/trident/state/map/MapState.java       |    26 -
 .../state/map/MicroBatchIBackingMap.java        |    85 -
 .../trident/state/map/NonTransactionalMap.java  |    67 -
 .../storm/trident/state/map/OpaqueMap.java      |   124 -
 .../trident/state/map/ReadOnlyMapState.java     |    26 -
 .../trident/state/map/RemovableMapState.java    |    25 -
 .../trident/state/map/SnapshottableMap.java     |    76 -
 .../trident/state/map/TransactionalMap.java     |   109 -
 .../state/snapshot/ReadOnlySnapshottable.java   |    24 -
 .../trident/state/snapshot/Snapshottable.java   |    27 -
 .../trident/testing/CountAsAggregator.java      |    47 -
 .../storm/trident/testing/FeederBatchSpout.java |   185 -
 .../testing/FeederCommitterBatchSpout.java      |    96 -
 .../storm/trident/testing/FixedBatchSpout.java  |    97 -
 .../apache/storm/trident/testing/IFeeder.java   |    23 -
 .../trident/testing/LRUMemoryMapState.java      |   154 -
 .../storm/trident/testing/MemoryBackingMap.java |    47 -
 .../storm/trident/testing/MemoryMapState.java   |   176 -
 .../org/apache/storm/trident/testing/Split.java |    36 -
 .../storm/trident/testing/StringLength.java     |    32 -
 .../storm/trident/testing/TrueFilter.java       |    30 -
 .../storm/trident/testing/TuplifyArgs.java      |    41 -
 .../storm/trident/topology/BatchInfo.java       |    33 -
 .../trident/topology/ITridentBatchBolt.java     |    32 -
 .../topology/MasterBatchCoordinator.java        |   316 -
 .../trident/topology/TransactionAttempt.java    |    66 -
 .../trident/topology/TridentBoltExecutor.java   |   439 -
 .../topology/TridentTopologyBuilder.java        |   751 -
 .../state/RotatingTransactionalState.java       |   186 -
 .../topology/state/TestTransactionalState.java  |    47 -
 .../topology/state/TransactionalState.java      |   191 -
 .../apache/storm/trident/tuple/ComboList.java   |    92 -
 .../apache/storm/trident/tuple/ConsList.java    |    60 -
 .../storm/trident/tuple/TridentTuple.java       |    34 -
 .../storm/trident/tuple/TridentTupleView.java   |   361 -
 .../storm/trident/tuple/ValuePointer.java       |    60 -
 .../storm/trident/util/ErrorEdgeFactory.java    |    28 -
 .../apache/storm/trident/util/IndexedEdge.java  |    50 -
 .../org/apache/storm/trident/util/LRUMap.java   |    35 -
 .../apache/storm/trident/util/TridentUtils.java |   117 -
 .../windowing/AbstractTridentWindowManager.java |   238 -
 .../windowing/ITridentWindowManager.java        |    59 -
 .../windowing/InMemoryTridentWindowManager.java |    72 -
 .../trident/windowing/InMemoryWindowsStore.java |   200 -
 .../windowing/InMemoryWindowsStoreFactory.java  |    44 -
 .../StoreBasedTridentWindowManager.java         |   217 -
 .../trident/windowing/TridentBatchTuple.java    |    42 -
 .../trident/windowing/WindowKryoSerializer.java |    87 -
 .../windowing/WindowTridentProcessor.java       |   265 -
 .../storm/trident/windowing/WindowsState.java   |    52 -
 .../trident/windowing/WindowsStateFactory.java  |    40 -
 .../trident/windowing/WindowsStateUpdater.java  |    81 -
 .../storm/trident/windowing/WindowsStore.java   |    79 -
 .../trident/windowing/WindowsStoreFactory.java  |    40 -
 .../windowing/config/BaseWindowConfig.java      |    48 -
 .../windowing/config/SlidingCountWindow.java    |    43 -
 .../windowing/config/SlidingDurationWindow.java |    44 -
 .../windowing/config/TumblingCountWindow.java   |    43 -
 .../config/TumblingDurationWindow.java          |    42 -
 .../trident/windowing/config/WindowConfig.java  |    57 -
 .../windowing/strategy/BaseWindowStrategy.java  |    32 -
 .../strategy/SlidingCountWindowStrategy.java    |    59 -
 .../strategy/SlidingDurationWindowStrategy.java |    60 -
 .../strategy/TumblingCountWindowStrategy.java   |    60 -
 .../TumblingDurationWindowStrategy.java         |    60 -
 .../windowing/strategy/WindowStrategy.java      |    45 -
 .../org/apache/storm/tuple/AddressedTuple.java  |    48 -
 .../src/jvm/org/apache/storm/tuple/Fields.java  |   142 -
 .../src/jvm/org/apache/storm/tuple/ITuple.java  |   214 -
 .../jvm/org/apache/storm/tuple/MessageId.java   |    94 -
 .../src/jvm/org/apache/storm/tuple/Tuple.java   |    68 -
 .../jvm/org/apache/storm/tuple/TupleImpl.java   |   356 -
 .../src/jvm/org/apache/storm/tuple/Values.java  |    37 -
 .../src/jvm/org/apache/storm/ui/UIHelpers.java  |     8 +-
 .../jvm/org/apache/storm/utils/BatchHelper.java |    92 -
 .../storm/utils/BufferFileInputStream.java      |    54 -
 .../apache/storm/utils/BufferInputStream.java   |    54 -
 .../apache/storm/utils/CRC32OutputStream.java   |    44 -
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   536 -
 .../jvm/org/apache/storm/utils/DRPCClient.java  |    66 -
 .../utils/DisruptorBackpressureCallback.java    |    27 -
 .../org/apache/storm/utils/DisruptorQueue.java  |   589 -
 .../storm/utils/ExtendedThreadPoolExecutor.java |    67 -
 .../jvm/org/apache/storm/utils/IPredicate.java  |    27 -
 .../storm/utils/IndifferentAccessMap.java       |   177 -
 .../org/apache/storm/utils/InprocMessaging.java |   100 -
 .../storm/utils/KeyedRoundRobinQueue.java       |    68 -
 .../org/apache/storm/utils/ListDelegate.java    |   156 -
 .../jvm/org/apache/storm/utils/LocalState.java  |   277 -
 .../jvm/org/apache/storm/utils/MutableInt.java  |    43 -
 .../jvm/org/apache/storm/utils/MutableLong.java |    43 -
 .../org/apache/storm/utils/MutableObject.java   |    38 -
 .../org/apache/storm/utils/NimbusClient.java    |   139 -
 .../utils/NimbusLeaderNotFoundException.java    |    41 -
 .../storm/utils/RegisteredGlobalState.java      |    62 -
 .../jvm/org/apache/storm/utils/RotatingMap.java |   128 -
 .../org/apache/storm/utils/ServiceRegistry.java |    47 -
 .../storm/utils/ShellBoltMessageQueue.java      |   121 -
 .../org/apache/storm/utils/ShellProcess.java    |   215 -
 .../jvm/org/apache/storm/utils/ShellUtils.java  |   505 -
 .../StormBoundedExponentialBackoffRetry.java    |    79 -
 .../storm/utils/StormCommonInstaller.java       |    43 -
 .../utils/StormConnectionStateConverter.java    |    44 -
 .../apache/storm/utils/ThriftTopologyUtils.java |    70 -
 .../src/jvm/org/apache/storm/utils/Time.java    |   235 -
 .../org/apache/storm/utils/TimeCacheMap.java    |   125 -
 .../apache/storm/utils/TopologySpoutLag.java    |     2 +
 .../org/apache/storm/utils/TransferDrainer.java |   140 -
 .../jvm/org/apache/storm/utils/TupleUtils.java  |    67 -
 .../src/jvm/org/apache/storm/utils/Utils.java   |  2320 -
 .../jvm/org/apache/storm/utils/VersionInfo.java |   131 -
 .../org/apache/storm/utils/VersionedStore.java  |   188 -
 .../jvm/org/apache/storm/utils/WebAppUtils.java |    31 +
 .../storm/utils/WindowedTimeThrottler.java      |    51 -
 .../storm/utils/WorkerBackpressureCallback.java |    26 -
 .../storm/utils/WorkerBackpressureThread.java   |    80 -
 .../org/apache/storm/utils/WritableUtils.java   |   375 -
 .../apache/storm/utils/ZookeeperAuthInfo.java   |    53 -
 .../storm/utils/ZookeeperServerCnxnFactory.java |    84 -
 .../storm/validation/ConfigValidation.java      |   716 -
 .../validation/ConfigValidationAnnotations.java |   218 -
 .../storm/validation/ConfigValidationUtils.java |   175 -
 .../storm/windowing/CountEvictionPolicy.java    |    81 -
 .../storm/windowing/CountTriggerPolicy.java     |    76 -
 .../storm/windowing/DefaultEvictionContext.java |    53 -
 .../jvm/org/apache/storm/windowing/Event.java   |    49 -
 .../org/apache/storm/windowing/EventImpl.java   |    51 -
 .../apache/storm/windowing/EvictionContext.java |    47 -
 .../apache/storm/windowing/EvictionPolicy.java  |    82 -
 .../storm/windowing/TimeEvictionPolicy.java     |    72 -
 .../storm/windowing/TimeTriggerPolicy.java      |   132 -
 .../storm/windowing/TimestampExtractor.java     |    35 -
 .../apache/storm/windowing/TriggerHandler.java  |    31 -
 .../apache/storm/windowing/TriggerPolicy.java   |    49 -
 .../org/apache/storm/windowing/TupleWindow.java |    26 -
 .../apache/storm/windowing/TupleWindowImpl.java |    93 -
 .../apache/storm/windowing/WaterMarkEvent.java  |    38 -
 .../windowing/WaterMarkEventGenerator.java      |   121 -
 .../windowing/WatermarkCountEvictionPolicy.java |    68 -
 .../windowing/WatermarkCountTriggerPolicy.java  |    91 -
 .../windowing/WatermarkTimeEvictionPolicy.java  |    77 -
 .../windowing/WatermarkTimeTriggerPolicy.java   |   126 -
 .../jvm/org/apache/storm/windowing/Window.java  |    55 -
 .../windowing/WindowLifecycleListener.java      |    43 -
 .../apache/storm/windowing/WindowManager.java   |   289 -
 .../storm/zookeeper/LeaderElectorImp.java       |   124 -
 .../apache/storm/zookeeper/ZkEventTypes.java    |    43 -
 .../apache/storm/zookeeper/ZkKeeperStates.java  |    41 -
 .../org/apache/storm/zookeeper/Zookeeper.java   |   545 -
 storm-core/src/py/__init__.py                   |    18 -
 storm-core/src/py/storm/DistributedRPC-remote   |   112 -
 storm-core/src/py/storm/DistributedRPC.py       |   311 -
 .../py/storm/DistributedRPCInvocations-remote   |   133 -
 .../src/py/storm/DistributedRPCInvocations.py   |   886 -
 storm-core/src/py/storm/Nimbus-remote           |   420 -
 storm-core/src/py/storm/Nimbus.py               |  9915 ----
 storm-core/src/py/storm/__init__.py             |    19 -
 storm-core/src/py/storm/constants.py            |    29 -
 storm-core/src/py/storm/ttypes.py               | 11695 ----
 storm-core/src/py_license_header.txt            |    18 -
 storm-core/src/storm.thrift                     |   775 -
 .../org/apache/storm/integration_test.clj       |     6 +-
 .../org/apache/storm/testing4j_test.clj         |     8 +-
 .../test/clj/org/apache/storm/cluster_test.clj  |    14 +-
 .../test/clj/org/apache/storm/drpc_test.clj     |    21 +-
 .../test/clj/org/apache/storm/grouping_test.clj |    28 +-
 .../clj/org/apache/storm/logviewer_test.clj     |     2 +-
 .../apache/storm/messaging/netty_unit_test.clj  |    36 +-
 .../test/clj/org/apache/storm/metrics_test.clj  |    11 +-
 .../test/clj/org/apache/storm/nimbus_test.clj   |    18 +-
 .../apache/storm/security/auth/auth_test.clj    |    17 +-
 .../storm/security/auth/nimbus_auth_test.clj    |    18 +-
 .../clj/org/apache/storm/trident/state_test.clj |     3 +-
 .../jvm/org/apache/storm/MessagingTest.java     |    68 -
 .../storm/PaceMakerStateStorageFactoryTest.java |   155 -
 .../jvm/org/apache/storm/PacemakerTest.java     |   242 -
 .../test/jvm/org/apache/storm/TestCgroups.java  |   129 -
 .../org/apache/storm/TestConfigValidate.java    |   801 -
 .../jvm/org/apache/storm/TickTupleTest.java     |   102 -
 .../apache/storm/blobstore/BlobStoreTest.java   |   469 -
 .../storm/blobstore/BlobSynchronizerTest.java   |   137 -
 .../storm/blobstore/ClientBlobStoreTest.java    |   179 -
 .../jvm/org/apache/storm/bolt/TestJoinBolt.java |   357 -
 .../cluster/StormClusterStateImplTest.java      |   116 -
 .../org/apache/storm/daemon/drpc/DRPCTest.java  |   252 -
 .../daemon/supervisor/BasicContainerTest.java   |   485 -
 .../storm/daemon/supervisor/ContainerTest.java  |   269 -
 .../storm/daemon/supervisor/SlotTest.java       |   498 -
 .../daemon/worker/LogConfigManagerTest.java     |   202 -
 .../apache/storm/daemon/worker/WorkerTest.java  |    39 -
 .../DependencyPropertiesParserTest.java         |    72 -
 .../dependency/DependencyUploaderTest.java      |   297 -
 .../storm/executor/error/ReportErrorTest.java   |    74 -
 .../storm/grouping/PartialKeyGroupingTest.java  |    66 -
 .../storm/grouping/ShuffleGroupingTest.java     |   147 -
 .../storm/localizer/AsyncLocalizerTest.java     |   187 -
 .../LocalizedResourceRetentionSetTest.java      |    85 -
 .../localizer/LocalizedResourceSetTest.java     |    74 -
 .../apache/storm/localizer/LocalizerTest.java   |   672 -
 .../org/apache/storm/localizer/localtest.zip    |   Bin 6378 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.jar    |   Bin 6591 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.tar    |   Bin 24576 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.tar.gz |   Bin 6106 -> 0 bytes
 .../storm/localizer/localtestwithsymlink.tgz    |   Bin 6106 -> 0 bytes
 .../ClusterMetricsConsumerExecutorTest.java     |   133 -
 .../metric/filter/FilterByMetricNameTest.java   |    95 -
 .../metric/internal/CountStatAndMetricTest.java |    86 -
 .../internal/LatencyStatAndMetricTest.java      |    83 -
 .../storm/metric/internal/RateTrackerTest.java  |    94 -
 .../metric/util/DataPointExpanderTest.java      |   109 -
 .../apache/storm/nimbus/LocalNimbusTest.java    |   132 -
 .../resource/TestResourceAwareScheduler.java    |  1366 -
 .../storm/scheduler/resource/TestUser.java      |   111 -
 .../TestUtilsForResourceAwareScheduler.java     |   369 -
 .../eviction/TestDefaultEvictionStrategy.java   |   751 -
 .../TestDefaultResourceAwareStrategy.java       |   331 -
 .../storm/security/auth/AuthUtilsTest.java      |   240 -
 .../storm/security/auth/AuthUtilsTestMock.java  |    82 -
 .../apache/storm/security/auth/AutoSSLTest.java |   136 -
 .../auth/DefaultHttpCredentialsPluginTest.java  |    90 -
 .../storm/security/auth/ReqContextTest.java     |    87 -
 .../security/auth/SaslTransportPluginTest.java  |    49 -
 .../storm/security/auth/ThriftClientTest.java   |    68 -
 .../storm/security/auth/ThriftServerTest.java   |    38 -
 .../authorizer/DRPCSimpleACLAuthorizerTest.java |   163 -
 .../BlowfishTupleSerializerTest.java            |    86 -
 .../GzipBridgeSerializationDelegateTest.java    |    82 -
 ...ipBridgeThriftSerializationDelegateTest.java |    71 -
 .../serialization/SerializationFactoryTest.java |    61 -
 .../storm/serialization/SerializationTest.java  |     7 +-
 .../ThriftBridgeSerializationDelegateTest.java  |    60 -
 .../apache/storm/spout/CheckpointSpoutTest.java |   262 -
 .../storm/state/InMemoryKeyValueStateTest.java  |    99 -
 .../apache/storm/streams/ProcessorBoltTest.java |   174 -
 .../streams/StatefulProcessorBoltTest.java      |   111 -
 .../apache/storm/streams/StreamBuilderTest.java |   268 -
 .../streams/WindowedProcessorBoltTest.java      |   110 -
 .../streams/processors/JoinProcessorTest.java   |   125 -
 .../topology/StatefulBoltExecutorTest.java      |   178 -
 .../StatefulWindowedBoltExecutorTest.java       |   131 -
 .../storm/topology/TopologyBuilderTest.java     |   118 -
 .../topology/WindowedBoltExecutorTest.java      |   230 -
 .../storm/trident/TestTridentTopology.java      |    76 -
 .../storm/trident/TridentWindowingTest.java     |   105 -
 .../jvm/org/apache/storm/tuple/FieldsTest.java  |   126 -
 .../utils/DisruptorQueueBackpressureTest.java   |   110 -
 .../apache/storm/utils/DisruptorQueueTest.java  |   187 -
 .../apache/storm/utils/MockTupleHelpers.java    |    40 -
 .../storm/utils/ShellBoltMessageQueueTest.java  |    85 -
 ...StormBoundedExponentialBackoffRetryTest.java |   101 -
 .../storm/utils/ThriftTopologyUtilsTest.java    |   106 -
 .../utils/ThrowableNestedCauseMatcher.java      |    44 -
 .../jvm/org/apache/storm/utils/TimeTest.java    |    95 -
 .../jvm/org/apache/storm/utils/UtilsTest.java   |   219 -
 .../utils/WorkerBackpressureThreadTest.java     |    50 -
 .../utils/ZookeeperServerCnxnFactoryTest.java   |    39 -
 .../staticmocking/ConfigUtilsInstaller.java     |    13 +-
 .../staticmocking/ReflectionUtilsInstaller.java |    39 +
 .../ServerConfigUtilsInstaller.java             |    41 +
 .../utils/staticmocking/UtilsInstaller.java     |    13 +-
 .../windowing/WaterMarkEventGeneratorTest.java  |   118 -
 .../storm/windowing/WindowManagerTest.java      |   570 -
 .../final-package/src/main/assembly/binary.xml  |     9 +-
 storm-dist/binary/pom.xml                       |     3 +-
 storm-dist/binary/storm-client-bin/pom.xml      |    65 +
 .../src/main/assembly/storm-client.xml          |    33 +
 storm-dist/binary/storm-drpc-server-bin/pom.xml |    64 -
 .../src/main/assembly/storm-drpc-server.xml     |    32 -
 storm-dist/binary/storm-webapp-bin/pom.xml      |    64 +
 .../src/main/assembly/storm-webapp.xml          |    32 +
 storm-drpc-server/pom.xml                       |   162 -
 .../apache/storm/daemon/drpc/DRPCServer.java    |   180 -
 .../webapp/AuthorizationExceptionMapper.java    |    39 -
 .../daemon/drpc/webapp/DRPCApplication.java     |    47 -
 .../daemon/drpc/webapp/DRPCExceptionMapper.java |    61 -
 .../storm/daemon/drpc/webapp/DRPCResource.java  |    63 -
 .../daemon/drpc/webapp/ReqContextFilter.java    |    69 -
 .../storm/daemon/drpc/DRPCServerTest.java       |   214 -
 storm-server/pom.xml                            |   157 +
 .../java/org/apache/storm/DaemonConfig.java     |   916 +
 .../ILocalClusterTrackedTopologyAware.java      |    54 +
 .../java/org/apache/storm/LocalCluster.java     |   834 +
 .../main/java/org/apache/storm/LocalDRPC.java   |    90 +
 .../java/org/apache/storm/ProcessSimulator.java |    97 +
 .../src/main/java/org/apache/storm/Testing.java |   716 +
 .../storm/blobstore/BlobKeySequenceInfo.java    |    40 +
 .../apache/storm/blobstore/BlobStoreUtils.java  |   270 +
 .../storm/blobstore/BlobSynchronizer.java       |   131 +
 .../storm/blobstore/FileBlobStoreImpl.java      |   269 +
 .../storm/blobstore/KeySequenceNumber.java      |   237 +
 .../storm/blobstore/LocalFsBlobStore.java       |   351 +
 .../storm/blobstore/LocalFsBlobStoreFile.java   |   159 +
 .../container/ResourceIsolationInterface.java   |    76 +
 .../storm/container/cgroup/CgroupManager.java   |   224 +
 .../daemon/drpc/BlockingOutstandingRequest.java |    67 +
 .../java/org/apache/storm/daemon/drpc/DRPC.java |   215 +
 .../apache/storm/daemon/drpc/DRPCThrift.java    |    58 +
 .../storm/daemon/drpc/OutstandingRequest.java   |    58 +
 .../storm/daemon/drpc/RequestFactory.java       |    24 +
 .../storm/daemon/metrics/MetricsUtils.java      |   109 +
 .../reporters/ConsolePreparableReporter.java    |    76 +
 .../reporters/CsvPreparableReporter.java        |    80 +
 .../reporters/JmxPreparableReporter.java        |    71 +
 .../metrics/reporters/PreparableReporter.java   |    32 +
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |  3807 ++
 .../storm/daemon/nimbus/TopologyActions.java    |    31 +
 .../storm/daemon/nimbus/TopologyResources.java  |    63 +
 .../daemon/nimbus/TopologyStateTransition.java  |    27 +
 .../storm/daemon/supervisor/BasicContainer.java |   723 +
 .../supervisor/BasicContainerLauncher.java      |    62 +
 .../storm/daemon/supervisor/Container.java      |   567 +
 .../daemon/supervisor/ContainerLauncher.java    |   106 +
 .../supervisor/ContainerRecoveryException.java  |    29 +
 .../DefaultUncaughtExceptionHandler.java        |    31 +
 .../supervisor/EventManagerPushCallback.java    |    37 +
 .../storm/daemon/supervisor/Killable.java       |    50 +
 .../storm/daemon/supervisor/LocalContainer.java |    94 +
 .../supervisor/LocalContainerLauncher.java      |    60 +
 .../daemon/supervisor/ReadClusterState.java     |   358 +
 .../daemon/supervisor/RunAsUserContainer.java   |   101 +
 .../supervisor/RunAsUserContainerLauncher.java  |    60 +
 .../apache/storm/daemon/supervisor/Slot.java    |   787 +
 .../daemon/supervisor/StandaloneSupervisor.java |    91 +
 .../storm/daemon/supervisor/Supervisor.java     |   362 +
 .../daemon/supervisor/SupervisorUtils.java      |   191 +
 .../apache/storm/daemon/supervisor/UniFunc.java |    22 +
 .../supervisor/timer/SupervisorHealthCheck.java |    41 +
 .../supervisor/timer/SupervisorHeartbeat.java   |    88 +
 .../daemon/supervisor/timer/UpdateBlobs.java    |   105 +
 .../org/apache/storm/event/EventManager.java    |    24 +
 .../org/apache/storm/event/EventManagerImp.java |   100 +
 .../apache/storm/healthcheck/HealthChecker.java |   122 +
 .../apache/storm/localizer/AsyncLocalizer.java  |   437 +
 .../org/apache/storm/localizer/ILocalizer.java  |    70 +
 .../localizer/LocalDownloadedResource.java      |   146 +
 .../apache/storm/localizer/LocalResource.java   |    44 +
 .../storm/localizer/LocalizedResource.java      |   130 +
 .../LocalizedResourceRetentionSet.java          |   140 +
 .../storm/localizer/LocalizedResourceSet.java   |   101 +
 .../org/apache/storm/localizer/Localizer.java   |   704 +
 .../storm/logging/ThriftAccessLogger.java       |    56 +
 .../logging/filters/AccessLoggingFilter.java    |    52 +
 .../metric/ClusterMetricsConsumerExecutor.java  |    89 +
 .../metric/LoggingClusterMetricsConsumer.java   |    86 +
 .../storm/metric/StormMetricsRegistry.java      |    88 +
 .../org/apache/storm/metric/api/DataPoint.java  |    43 +
 .../metric/api/IClusterMetricsConsumer.java     |    63 +
 .../storm/nimbus/DefaultTopologyValidator.java  |    31 +
 .../org/apache/storm/nimbus/ILeaderElector.java |    73 +
 .../nimbus/ITopologyActionNotifierPlugin.java   |    43 +
 .../apache/storm/nimbus/ITopologyValidator.java |    28 +
 .../storm/pacemaker/IServerMessageHandler.java  |    25 +
 .../org/apache/storm/pacemaker/Pacemaker.java   |   218 +
 .../apache/storm/pacemaker/PacemakerServer.java |   169 +
 .../pacemaker/codec/ThriftNettyServerCodec.java |   100 +
 .../storm/scheduler/DefaultScheduler.java       |   111 +
 .../apache/storm/scheduler/EvenScheduler.java   |   172 +
 .../storm/scheduler/IsolationScheduler.java     |   418 +
 .../scheduler/multitenant/DefaultPool.java      |   219 +
 .../storm/scheduler/multitenant/FreePool.java   |   125 +
 .../scheduler/multitenant/IsolatedPool.java     |   371 +
 .../multitenant/MultitenantScheduler.java       |    99 +
 .../storm/scheduler/multitenant/Node.java       |   340 +
 .../storm/scheduler/multitenant/NodePool.java   |   296 +
 .../resource/ResourceAwareScheduler.java        |   437 +
 .../eviction/DefaultEvictionStrategy.java       |   126 +
 .../strategies/eviction/IEvictionStrategy.java  |    42 +
 .../DefaultSchedulingPriorityStrategy.java      |    80 +
 .../priority/ISchedulingPriorityStrategy.java   |    36 +
 .../storm/testing/CompleteTopologyParam.java    |   103 +
 .../storm/testing/InProcessZookeeper.java       |    59 +
 .../java/org/apache/storm/testing/TestJob.java  |    43 +
 .../apache/storm/testing/TrackedTopology.java   |   135 +
 .../storm/topology/ConfigurableTopology.java    |   184 +
 .../apache/storm/utils/BufferInputStream.java   |    54 +
 .../apache/storm/utils/ServerConfigUtils.java   |   199 +
 .../org/apache/storm/utils/ServerUtils.java     |   772 +
 .../storm/utils/StormCommonInstaller.java       |    43 +
 .../storm/zookeeper/LeaderElectorImp.java       |   123 +
 .../org/apache/storm/zookeeper/Zookeeper.java   |   261 +
 .../java/org/apache/storm/MessagingTest.java    |    68 +
 .../java/org/apache/storm/PacemakerTest.java    |   242 +
 .../test/java/org/apache/storm/TestCgroups.java |   129 +
 .../apache/storm/TestDaemonConfigValidate.java  |   110 +
 .../java/org/apache/storm/TickTupleTest.java    |   102 +
 .../apache/storm/blobstore/BlobStoreTest.java   |   469 +
 .../storm/blobstore/BlobSynchronizerTest.java   |   137 +
 .../org/apache/storm/daemon/drpc/DRPCTest.java  |   253 +
 .../daemon/supervisor/BasicContainerTest.java   |   486 +
 .../storm/daemon/supervisor/ContainerTest.java  |   270 +
 .../storm/daemon/supervisor/SlotTest.java       |   498 +
 .../storm/localizer/AsyncLocalizerTest.java     |   187 +
 .../LocalizedResourceRetentionSetTest.java      |    85 +
 .../localizer/LocalizedResourceSetTest.java     |    74 +
 .../apache/storm/localizer/LocalizerTest.java   |   679 +
 .../ClusterMetricsConsumerExecutorTest.java     |   133 +
 .../apache/storm/nimbus/LocalNimbusTest.java    |   132 +
 .../resource/TestResourceAwareScheduler.java    |  1367 +
 .../storm/scheduler/resource/TestUser.java      |   111 +
 .../TestUtilsForResourceAwareScheduler.java     |   369 +
 .../eviction/TestDefaultEvictionStrategy.java   |   752 +
 .../TestDefaultResourceAwareStrategy.java       |   332 +
 .../src/test/resources/localizer/localtest.zip  |   Bin 0 -> 6378 bytes
 .../localizer/localtestwithsymlink.jar          |   Bin 0 -> 6591 bytes
 .../localizer/localtestwithsymlink.tar          |   Bin 0 -> 24576 bytes
 .../localizer/localtestwithsymlink.tar.gz       |   Bin 0 -> 6106 bytes
 .../localizer/localtestwithsymlink.tgz          |   Bin 0 -> 6106 bytes
 storm-submit-tools/pom.xml                      |     4 +-
 storm-webapp/pom.xml                            |   163 +
 .../apache/storm/daemon/drpc/DRPCServer.java    |   182 +
 .../webapp/AuthorizationExceptionMapper.java    |    39 +
 .../daemon/drpc/webapp/DRPCApplication.java     |    47 +
 .../daemon/drpc/webapp/DRPCExceptionMapper.java |    61 +
 .../storm/daemon/drpc/webapp/DRPCResource.java  |    63 +
 .../daemon/drpc/webapp/ReqContextFilter.java    |    69 +
 .../storm/daemon/drpc/DRPCServerTest.java       |   214 +
 2437 files changed, 266439 insertions(+), 265276 deletions(-)
----------------------------------------------------------------------



[47/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java
new file mode 100644
index 0000000..78468e0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -0,0 +1,1697 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import org.apache.storm.scheduler.resource.strategies.scheduling.IStrategy;
+import org.apache.storm.serialization.IKryoDecorator;
+import org.apache.storm.serialization.IKryoFactory;
+import org.apache.storm.validation.ConfigValidation;
+import org.apache.storm.validation.ConfigValidationAnnotations.*;
+import org.apache.storm.validation.ConfigValidation.*;
+import com.esotericsoftware.kryo.Serializer;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+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
+ * serializations.
+ *
+ * This class also provides constants for all the configurations possible on
+ * a Storm cluster and Storm topology. Each constant is paired with an annotation
+ * that defines the validity criterion of the corresponding field. Default
+ * values for these configs can be found in defaults.yaml.
+ *
+ * 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
+ * Spouts.
+ */
+public class Config extends HashMap<String, Object> {
+
+    //DO NOT CHANGE UNLESS WE ADD IN STATE NOT STORED IN THE PARENT CLASS
+    private static final long serialVersionUID = -1550278723792864455L;
+
+    /**
+     * The serializer class for ListDelegate (tuple payload).
+     * The default serializer will be ListDelegateSerializer
+     */
+    @isString
+    public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer";
+
+    /**
+     * Disable load aware grouping support.
+     */
+    @isBoolean
+    @NotNull
+    public static final String TOPOLOGY_DISABLE_LOADAWARE_MESSAGING = "topology.disable.loadaware.messaging";
+
+    /**
+     * Try to serialize all tuples, even for local transfers.  This should only be used
+     * for testing, as a sanity check that all of your tuples are setup properly.
+     */
+    @isBoolean
+    public static final String TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE = "topology.testing.always.try.serialize";
+
+    /**
+     * A map with blobstore keys mapped to each filename the worker will have access to in the
+     * launch directory to the blob by local file name and uncompress flag. Both localname and
+     * uncompress flag are optional. It uses the key is localname is not specified. Each topology
+     * will have different map of blobs.  Example: topology.blobstore.map: {"blobstorekey" :
+     * {"localname": "myblob", "uncompress": false}, "blobstorearchivekey" :
+     * {"localname": "myarchive", "uncompress": true}}
+     */
+    @CustomValidator(validatorClass = MapOfStringToMapOfStringToObjectValidator.class)
+    public static final String TOPOLOGY_BLOBSTORE_MAP = "topology.blobstore.map";
+
+    /**
+     * How often a worker should check dynamic log level timeouts for expiration.
+     * For expired logger settings, the clean up polling task will reset the log levels
+     * to the original levels (detected at startup), and will clean up the timeout map
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String WORKER_LOG_LEVEL_RESET_POLL_SECS = "worker.log.level.reset.poll.secs";
+
+    /**
+     * How often a task should sync credentials, worst case.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TASK_CREDENTIALS_POLL_SECS = "task.credentials.poll.secs";
+
+    /**
+     * How often to poll for changed topology backpressure flag from ZK
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TASK_BACKPRESSURE_POLL_SECS = "task.backpressure.poll.secs";
+
+    /**
+     * Whether to enable backpressure in for a certain topology
+     */
+    @isBoolean
+    public static final String TOPOLOGY_BACKPRESSURE_ENABLE = "topology.backpressure.enable";
+
+    /**
+     * This signifies the tuple congestion in a disruptor queue.
+     * When the used ratio of a disruptor queue is higher than the high watermark,
+     * the backpressure scheme, if enabled, should slow down the tuple sending speed of
+     * the spouts until reaching the low watermark.
+     */
+    @isPositiveNumber
+    public static final String BACKPRESSURE_DISRUPTOR_HIGH_WATERMARK="backpressure.disruptor.high.watermark";
+
+    /**
+     * This signifies a state that a disruptor queue has left the congestion.
+     * If the used ratio of a disruptor queue is lower than the low watermark,
+     * it will unset the backpressure flag.
+     */
+    @isPositiveNumber
+    public static final String BACKPRESSURE_DISRUPTOR_LOW_WATERMARK="backpressure.disruptor.low.watermark";
+
+    /**
+     * A list of users that are allowed to interact with the topology.  To use this set
+     * nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
+     */
+    @isStringList
+    public static final String TOPOLOGY_USERS = "topology.users";
+
+    /**
+     * A list of groups that are allowed to interact with the topology.  To use this set
+     * nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
+     */
+    @isStringList
+    public static final String TOPOLOGY_GROUPS = "topology.groups";
+
+    /**
+     * True if Storm should timeout messages or not. Defaults to true. This is meant to be used
+     * in unit tests to prevent tuples from being accidentally timed out during the test.
+     */
+    @isBoolean
+    public static final String TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS = "topology.enable.message.timeouts";
+
+    /**
+     * When set to true, Storm will log every message that's emitted.
+     */
+    @isBoolean
+    public static final String TOPOLOGY_DEBUG = "topology.debug";
+
+    /**
+     * The serializer for communication between shell components and non-JVM
+     * processes
+     */
+    @isString
+    public static final String TOPOLOGY_MULTILANG_SERIALIZER = "topology.multilang.serializer";
+
+    /**
+     * 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
+     * on each component in the topology to tune the performance of a topology.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_WORKERS = "topology.workers";
+
+    /**
+     * 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
+     * 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).
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_TASKS = "topology.tasks";
+
+    /**
+     * The maximum amount of memory an instance of a spout/bolt will take on heap. This enables the scheduler
+     * to allocate slots on machines with enough available memory. A default value will be set for this config if user does not override
+     */
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB = "topology.component.resources.onheap.memory.mb";
+
+    /**
+     * The maximum amount of memory an instance of a spout/bolt will take off heap. This enables the scheduler
+     * to allocate slots on machines with enough available memory.  A default value will be set for this config if user does not override
+     */
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB = "topology.component.resources.offheap.memory.mb";
+
+    /**
+     * The config indicates the percentage of cpu for a core an instance(executor) of a component will use.
+     * Assuming the a core value to be 100, a value of 10 indicates 10% of the core.
+     * The P in PCORE represents the term "physical".  A default value will be set for this config if user does not override
+     */
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT = "topology.component.cpu.pcore.percent";
+
+    /**
+     * The class name of the {@link org.apache.storm.state.StateProvider} implementation. If not specified
+     * defaults to {@link org.apache.storm.state.InMemoryKeyValueStateProvider}. This can be overridden
+     * at the component level.
+     */
+    @isString
+    public static final String TOPOLOGY_STATE_PROVIDER = "topology.state.provider";
+
+    /**
+     * The configuration specific to the {@link org.apache.storm.state.StateProvider} implementation.
+     * This can be overridden at the component level. The value and the interpretation of this config
+     * is based on the state provider implementation. For e.g. this could be just a config file name
+     * which contains the config for the state provider implementation.
+     */
+    @isString
+    public static final String TOPOLOGY_STATE_PROVIDER_CONFIG = "topology.state.provider.config";
+
+    /**
+     * Topology configuration to specify the checkpoint interval (in millis) at which the
+     * topology state is saved when {@link org.apache.storm.topology.IStatefulBolt} bolts are involved.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_STATE_CHECKPOINT_INTERVAL = "topology.state.checkpoint.interval.ms";
+
+    /**
+     * A per topology config that specifies the maximum amount of memory a worker can use for that specific topology
+     */
+    @isPositiveNumber
+    public static final String TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB = "topology.worker.max.heap.size.mb";
+
+    /**
+     * The strategy to use when scheduling a topology with Resource Aware Scheduler
+     */
+    @NotNull
+    @isImplementationOfClass(implementsClass = IStrategy.class)
+    public static final String TOPOLOGY_SCHEDULER_STRATEGY = "topology.scheduler.strategy";
+
+    /**
+     * How many executors to spawn for ackers.
+     *
+     * <p>By not setting this variable or setting it as null, Storm will set the number of acker executors
+     * to be equal to the number of workers configured for this topology. If this variable is set to 0,
+     * then Storm will immediately ack tuples as soon as they come off the spout, effectively disabling reliability.</p>
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_ACKER_EXECUTORS = "topology.acker.executors";
+
+    /**
+     * How many executors to spawn for event logger.
+     *
+     * <p>By not setting this variable or setting it as null, Storm will set the number of eventlogger executors
+     * to be equal to the number of workers configured for this topology. If this variable is set to 0,
+     * event logging will be disabled.</p>
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_EVENTLOGGER_EXECUTORS = "topology.eventlogger.executors";
+
+    /**
+     * The maximum amount of time given to the topology to fully process a message
+     * emitted by a spout. If the message is not acked within this time frame, Storm
+     * will fail the message on the spout. Some spouts implementations will then replay
+     * the message at a later time.
+     */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
+    public static final String TOPOLOGY_MESSAGE_TIMEOUT_SECS = "topology.message.timeout.secs";
+
+    /**
+     * A list of serialization registrations for Kryo ( https://github.com/EsotericSoftware/kryo ),
+     * the underlying serialization framework for Storm. A serialization can either
+     * be the name of a class (in which case Kryo will automatically create a serializer for the class
+     * that saves all the object's fields), or an implementation of com.esotericsoftware.kryo.Serializer.
+     *
+     * See Kryo's documentation for more information about writing custom serializers.
+     */
+    @isKryoReg
+    public static final String TOPOLOGY_KRYO_REGISTER = "topology.kryo.register";
+
+    /**
+     * 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
+     * is called with storm's kryo instance as the only argument.
+     */
+    @isStringList
+    public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators";
+
+    /**
+     * Class that specifies how to create a Kryo instance for serialization. Storm will then apply
+     * topology.kryo.register and topology.kryo.decorators on top of this. The default implementation
+     * implements topology.fall.back.on.java.serialization and turns references off.
+     */
+    @isString
+    public static final String TOPOLOGY_KRYO_FACTORY = "topology.kryo.factory";
+
+    /**
+     * Whether or not Storm should skip the loading of kryo registrations for which it
+     * does not know the class or have the serializer implementation. Otherwise, the task will
+     * fail to load and will throw an error at runtime. The use case of this is if you want to
+     * declare your serializations on the storm.yaml files on the cluster rather than every single
+     * time you submit a topology. Different applications may use different serializations and so
+     * a single application may not have the code for the other serializers used by other apps.
+     * By setting this config to true, Storm will ignore that it doesn't have those other serializations
+     * rather than throw an error.
+     */
+    @isBoolean
+    public static final String TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS= "topology.skip.missing.kryo.registrations";
+
+    /**
+     * 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 maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable.
+     */
+
+    @isListEntryCustom(entryValidatorClasses={MetricRegistryValidator.class})
+    public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register";
+
+    /**
+     * A map of metric name to class name implementing IMetric that will be created once per worker JVM
+     */
+    @isMapEntryType(keyType = String.class, valueType = String.class)
+    public static final String TOPOLOGY_WORKER_METRICS = "topology.worker.metrics";
+
+    /**
+     * A map of metric name to class name implementing IMetric that will be created once per worker JVM
+     */
+    @isMapEntryType(keyType = String.class, valueType = String.class)
+    public static final String WORKER_METRICS = "worker.metrics";
+
+    /**
+     * The maximum parallelism allowed for a component in this topology. This configuration is
+     * typically used in testing to limit the number of threads spawned in local mode.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_MAX_TASK_PARALLELISM="topology.max.task.parallelism";
+
+    /**
+     * 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
+     * their tuples with a message id.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending";
+
+    /**
+     * 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
+     */
+    @isString
+    public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy";
+
+    /**
+     * The amount of milliseconds the SleepEmptyEmitStrategy should sleep for.
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS="topology.sleep.spout.wait.strategy.time.ms";
+
+    /**
+     * The maximum amount of time a component gives a source of state to synchronize before it requests
+     * synchronization again.
+     */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
+    public static final String TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS="topology.state.synchronization.timeout.secs";
+
+    /**
+     * The percentage of tuples to sample to produce stats for a task.
+     */
+    @isPositiveNumber
+    public static final String TOPOLOGY_STATS_SAMPLE_RATE="topology.stats.sample.rate";
+
+    /**
+     * The time period that builtin metrics data in bucketed into.
+     */
+    @isInteger
+    public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs";
+
+    /**
+     * Whether or not to use Java serialization in a topology.
+     */
+    @isBoolean
+    public static final String TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION="topology.fall.back.on.java.serialization";
+
+    /**
+     * Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS.
+     */
+    @isStringOrStringList
+    public static final String TOPOLOGY_WORKER_CHILDOPTS="topology.worker.childopts";
+
+    /**
+     * Topology-specific options GC for the worker child process. This overrides WORKER_GC_CHILDOPTS.
+     */
+    @isStringOrStringList
+    public static final String TOPOLOGY_WORKER_GC_CHILDOPTS="topology.worker.gc.childopts";
+
+    /**
+     * Topology-specific options for the logwriter process of a worker.
+     */
+    @isStringOrStringList
+    public static final String TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS="topology.worker.logwriter.childopts";
+
+    /**
+     * Topology-specific classpath for the worker child process. This is combined to the usual classpath.
+     */
+    @isStringOrStringList
+    public static final String TOPOLOGY_CLASSPATH="topology.classpath";
+
+    /**
+     * Topology-specific classpath for the worker child process. This will be *prepended* to
+     * the usual classpath, meaning it can override the Storm classpath. This is for debugging
+     * purposes, and is disabled by default. To allow topologies to be submitted with user-first
+     * classpaths, set the storm.topology.classpath.beginning.enabled config to true.
+     */
+    @isStringOrStringList
+    public static final String TOPOLOGY_CLASSPATH_BEGINNING="topology.classpath.beginning";
+
+    /**
+     * Topology-specific environment variables for the worker child process.
+     * This is added to the existing environment (that of the supervisor)
+     */
+    @isMapEntryType(keyType = String.class, valueType = String.class)
+    public static final String TOPOLOGY_ENVIRONMENT="topology.environment";
+
+    /*
+     * Topology-specific option to disable/enable bolt's outgoing overflow buffer.
+     * Enabling this option ensures that the bolt can always clear the incoming messages,
+     * preventing live-lock for the topology with cyclic flow.
+     * The overflow buffer can fill degrading the performance gradually,
+     * eventually running out of memory.
+     */
+    @isBoolean
+    public static final String TOPOLOGY_BOLTS_OUTGOING_OVERFLOW_BUFFER_ENABLE="topology.bolts.outgoing.overflow.buffer.enable";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the window length as a count of number of tuples
+     * in the window.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT = "topology.bolts.window.length.count";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the window length in time duration.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS = "topology.bolts.window.length.duration.ms";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the sliding interval as a count of number of tuples.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT = "topology.bolts.window.sliding.interval.count";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the sliding interval in time duration.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS = "topology.bolts.window.sliding.interval.duration.ms";
+
+    /**
+     * Bolt-specific configuration for windowed bolts to specify the name of the stream on which late tuples are
+     * going to be emitted. This configuration should only be used from the BaseWindowedBolt.withLateTupleStream builder
+     * method, and not as global parameter, otherwise IllegalArgumentException is going to be thrown.
+     */
+    @isString
+    public static final String TOPOLOGY_BOLTS_LATE_TUPLE_STREAM = "topology.bolts.late.tuple.stream";
+
+    /**
+     * Bolt-specific configuration for windowed bolts to specify the maximum time lag of the tuple timestamp
+     * in milliseconds. It means that the tuple timestamps cannot be out of order by more than this amount.
+     * This config will be effective only if {@link org.apache.storm.windowing.TimestampExtractor} is specified.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_TUPLE_TIMESTAMP_MAX_LAG_MS = "topology.bolts.tuple.timestamp.max.lag.ms";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the time interval for generating
+     * watermark events. Watermark event tracks the progress of time when tuple timestamp is used.
+     * This config is effective only if {@link org.apache.storm.windowing.TimestampExtractor} is specified.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS = "topology.bolts.watermark.event.interval.ms";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the name of the field in the tuple that holds
+     * the message id. This is used to track the windowing boundaries and avoid re-evaluating the windows
+     * during recovery of IStatefulWindowedBolt
+     */
+    @isString
+    public static final String TOPOLOGY_BOLTS_MESSAGE_ID_FIELD_NAME = "topology.bolts.message.id.field.name";
+
+    /**
+     * This config is available for TransactionalSpouts, and contains the id ( a String) for
+     * the transactional topology. This id is used to store the state of the transactional
+     * topology in Zookeeper.
+     */
+    @isString
+    public static final String TOPOLOGY_TRANSACTIONAL_ID="topology.transactional.id";
+
+    /**
+     * 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
+     * monitoring system. These hooks are instantiated using the zero-arg constructor.
+     */
+    @isStringList
+    public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks";
+
+    /**
+     * The size of the Disruptor receive queue for each executor. Must be a power of 2.
+     */
+    @isPowerOf2
+    public static final String TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE="topology.executor.receive.buffer.size";
+
+    /**
+     * The size of the Disruptor send queue for each executor. Must be a power of 2.
+     */
+    @isPowerOf2
+    public static final String TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE="topology.executor.send.buffer.size";
+
+    /**
+     * The size of the Disruptor transfer queue for each worker.
+     */
+    @isInteger
+    @isPowerOf2
+    public static final String TOPOLOGY_TRANSFER_BUFFER_SIZE="topology.transfer.buffer.size";
+
+    /**
+     * How often a tick tuple from the "__system" component and "__tick" stream should be sent
+     * to tasks. Meant to be used as a component-specific configuration.
+     */
+    @isInteger
+    public static final String TOPOLOGY_TICK_TUPLE_FREQ_SECS="topology.tick.tuple.freq.secs";
+
+    /**
+     * @deprecated this is no longer supported
+     * Configure the wait strategy used for internal queuing. Can be used to tradeoff latency
+     * vs. throughput
+     */
+    @Deprecated
+    @isString
+    public static final String TOPOLOGY_DISRUPTOR_WAIT_STRATEGY="topology.disruptor.wait.strategy";
+
+    /**
+     * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
+     * via the TopologyContext.
+     */
+    @isInteger
+    public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size";
+
+    /**
+     * 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.
+     */
+    @isInteger
+    public static final String TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS="topology.error.throttle.interval.secs";
+
+    /**
+     * See doc for TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL="topology.max.error.report.per.interval";
+
+    /**
+     * How often a batch can be emitted in a Trident topology.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS="topology.trident.batch.emit.interval.millis";
+
+    /**
+     * Maximum number of tuples that can be stored inmemory cache in windowing operators for fast access without fetching
+     * them from store.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_TRIDENT_WINDOWING_INMEMORY_CACHE_LIMIT="topology.trident.windowing.cache.tuple.limit";
+
+    /**
+     * The id assigned to a running topology. The id is the storm name with a unique nonce appended.
+     */
+    @isString
+    public static final String STORM_ID = "storm.id";
+
+    /**
+     * Name of the topology. This config is automatically set by Storm when the topology is submitted.
+     */
+    @isString
+    public final static String TOPOLOGY_NAME="topology.name";
+
+    /**
+     * The principal who submitted a topology
+     */
+    @isString
+    public final static String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal";
+
+    /**
+     * The local user name of the user who submitted a topology.
+     */
+    @isString
+    public static final String TOPOLOGY_SUBMITTER_USER = "topology.submitter.user";
+
+    /**
+     * Array of components that scheduler should try to place on separate hosts.
+     */
+    @isStringList
+    public static final String TOPOLOGY_SPREAD_COMPONENTS = "topology.spread.components";
+
+    /**
+     * A list of IAutoCredentials that the topology should load and use.
+     */
+    @isStringList
+    public static final String TOPOLOGY_AUTO_CREDENTIALS = "topology.auto-credentials";
+
+    /**
+     * Max pending tuples in one ShellBolt
+     */
+    @NotNull
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_SHELLBOLT_MAX_PENDING="topology.shellbolt.max.pending";
+
+    /**
+     * How long a subprocess can go without heartbeating before the ShellSpout/ShellBolt tries to
+     * suicide itself.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_SUBPROCESS_TIMEOUT_SECS = "topology.subprocess.timeout.secs";
+
+    /**
+     * Topology central logging sensitivity to determine who has access to logs in central logging system.
+     * The possible values are:
+     *   S0 - Public (open to all users on grid)
+     *   S1 - Restricted
+     *   S2 - Confidential
+     *   S3 - Secret (default.)
+     */
+    @isString(acceptedValues = {"S0", "S1", "S2", "S3"})
+    public static final String TOPOLOGY_LOGGING_SENSITIVITY="topology.logging.sensitivity";
+
+    /**
+     * Sets the priority for a topology
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_PRIORITY = "topology.priority";
+
+    /**
+     * The root directory in ZooKeeper for metadata about TransactionalSpouts.
+     */
+    @isString
+    public static final String TRANSACTIONAL_ZOOKEEPER_ROOT="transactional.zookeeper.root";
+
+    /**
+     * The list of zookeeper servers in which to keep the transactional state. If null (which is default),
+     * will use storm.zookeeper.servers
+     */
+    @isStringList
+    public static final String TRANSACTIONAL_ZOOKEEPER_SERVERS="transactional.zookeeper.servers";
+
+    /**
+     * The port to use to connect to the transactional zookeeper servers. If null (which is default),
+     * will use storm.zookeeper.port
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TRANSACTIONAL_ZOOKEEPER_PORT="transactional.zookeeper.port";
+
+    /**
+     * The user as which the nimbus client should be acquired to perform the operation.
+     */
+    @isString
+    public static final String STORM_DO_AS_USER="storm.doAsUser";
+
+    /**
+     * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler
+     * to org.apache.storm.scheduler.multitenant.MultitenantScheduler
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
+
+    /**
+     * Configure timeout milliseconds used for disruptor queue wait strategy. Can be used to tradeoff latency
+     * vs. CPU usage
+     */
+    @isInteger
+    @NotNull
+    public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis";
+
+    /**
+     * The number of tuples to batch before sending to the next thread.  This number is just an initial suggestion and
+     * the code may adjust it as your topology runs.
+     */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
+    public static final String TOPOLOGY_DISRUPTOR_BATCH_SIZE="topology.disruptor.batch.size";
+
+    /**
+     * The maximum age in milliseconds a batch can be before being sent to the next thread.  This number is just an
+     * initial suggestion and the code may adjust it as your topology runs.
+     */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
+    public static final String TOPOLOGY_DISRUPTOR_BATCH_TIMEOUT_MILLIS="topology.disruptor.batch.timeout.millis";
+
+    /**
+     * Minimum number of nimbus hosts where the code must be replicated before leader nimbus
+     * is allowed to perform topology activation tasks like setting up heartbeats/assignments
+     * and marking the topology as active. default is 0.
+     */
+    @isNumber
+    public static final String TOPOLOGY_MIN_REPLICATION_COUNT = "topology.min.replication.count";
+
+    /**
+     * Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
+     * Once this time is elapsed nimbus will go ahead and perform topology activation tasks even
+     * if required nimbus.min.replication.count is not achieved. The default is 0 seconds, a value of
+     * -1 indicates to wait for ever.
+     */
+    @isNumber
+    public static final String TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC = "topology.max.replication.wait.time.sec";
+
+    /**
+     * This is a config that is not likely to be used.  Internally the disruptor queue will batch entries written
+     * into the queue.  A background thread pool will flush those batches if they get too old.  By default that
+     * pool can grow rather large, and sacrifice some CPU time to keep the latency low.  In some cases you may
+     * want the queue to be smaller so there is less CPU used, but the latency will increase in some situations.
+     * This configs is on a per cluster bases, if you want to control this on a per topology bases you need to set
+     * the java System property for the worker "num_flusher_pool_threads" to the value you want.
+     */
+    @isInteger
+    public static final String STORM_WORKER_DISRUPTOR_FLUSHER_MAX_POOL_SIZE = "storm.worker.disruptor.flusher.max.pool.size";
+
+    /**
+     * The list of servers that Pacemaker is running on.
+     */
+    @isStringList
+    public static final String PACEMAKER_SERVERS = "pacemaker.servers";
+
+    /**
+     * The port Pacemaker should run on. Clients should
+     * connect to this port to submit or read heartbeats.
+     */
+    @isNumber
+    @isPositiveNumber
+    public static final String PACEMAKER_PORT = "pacemaker.port";
+
+    /**
+     * This should be one of "DIGEST", "KERBEROS", or "NONE"
+     * Determines the mode of authentication the pacemaker server and client use.
+     * The client must either match the server, or be NONE. In the case of NONE,
+     * no authentication is performed for the client, and if the server is running with
+     * DIGEST or KERBEROS, the client can only write to the server (no reads).
+     * This is intended to provide a primitive form of access-control.
+     */
+    @CustomValidator(validatorClass=ConfigValidation.PacemakerAuthTypeValidator.class)
+    public static final String PACEMAKER_AUTH_METHOD = "pacemaker.auth.method";
+
+    /**
+     * Max no.of seconds group mapping service will cache user groups
+     */
+    @isInteger
+    public static final String STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS = "storm.group.mapping.service.cache.duration.secs";
+
+    /**
+     * List of DRPC servers so that the DRPCSpout knows who to talk to.
+     */
+    @isStringList
+    public static final String DRPC_SERVERS = "drpc.servers";
+
+    /**
+     * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
+
+    /**
+     * The number of times to retry a Nimbus operation.
+     */
+    @isNumber
+    public static final String STORM_NIMBUS_RETRY_TIMES="storm.nimbus.retry.times";
+
+    /**
+     * The starting interval between exponential backoff retries of a Nimbus operation.
+     */
+    @isNumber
+    public static final String STORM_NIMBUS_RETRY_INTERVAL="storm.nimbus.retry.interval.millis";
+
+    /**
+     * The ceiling of the interval between retries of a client connect to Nimbus operation.
+     */
+    @isNumber
+    public static final String STORM_NIMBUS_RETRY_INTERVAL_CEILING="storm.nimbus.retry.intervalceiling.millis";
+
+    /**
+     * The Nimbus transport plug-in for Thrift client/server communication
+     */
+    @isString
+    public static final String NIMBUS_THRIFT_TRANSPORT_PLUGIN = "nimbus.thrift.transport";
+
+    /**
+     * Which port the Thrift interface of Nimbus should run on. Clients should
+     * connect to this port to upload jars and submit topologies.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String NIMBUS_THRIFT_PORT = "nimbus.thrift.port";
+
+    /**
+     * Nimbus thrift server queue size, default is 100000. This is the request queue size , when there are more requests
+     * than number of threads to serve the requests, those requests will be queued to this queue. If the request queue
+     * size > this config, then the incoming requests will be rejected.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String NIMBUS_QUEUE_SIZE = "nimbus.queue.size";
+
+    /**
+     * The number of threads that should be used by the nimbus thrift server.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String NIMBUS_THRIFT_THREADS = "nimbus.thrift.threads";
+
+    /**
+     * The maximum buffer size thrift should use when reading messages.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size";
+
+    /**
+     * How long before a Thrift Client socket hangs before timeout
+     * and restart the socket.
+     */
+    @isInteger
+    public static final String STORM_THRIFT_SOCKET_TIMEOUT_MS = "storm.thrift.socket.timeout.ms";
+
+    /**
+     * The DRPC transport plug-in for Thrift client/server communication
+     */
+    @isString
+    public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport";
+
+    /**
+     * This port is used by Storm DRPC for receiving DPRC requests from clients.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String DRPC_PORT = "drpc.port";
+
+    /**
+     * DRPC thrift server queue size
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
+
+    /**
+     * DRPC thrift server worker threads
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
+
+    /**
+     * The maximum buffer size thrift should use when reading messages for DRPC.
+     */
+    @isNumber
+    @isPositiveNumber
+    public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size";
+
+    /**
+     * The DRPC invocations transport plug-in for Thrift client/server communication
+     */
+    @isString
+    public static final String DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN = "drpc.invocations.thrift.transport";
+
+    /**
+     * DRPC invocations thrift server worker threads
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String DRPC_INVOCATIONS_THREADS = "drpc.invocations.threads";
+
+    /**
+     * The default transport plug-in for Thrift client/server communication
+     */
+    @isString
+    public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport";
+
+    /**
+     * How long a worker can go without heartbeating before the supervisor tries to
+     * restart the worker process.
+     */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
+    public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
+
+    /**
+     * A list of hosts of ZooKeeper servers used to manage the cluster.
+     */
+    @isStringList
+    public static final String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers";
+
+    /**
+     * The port Storm will use to connect to each of the ZooKeeper servers.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String STORM_ZOOKEEPER_PORT = "storm.zookeeper.port";
+
+    /**
+     * This is part of a temporary workaround to a ZK bug, it is the 'scheme:acl' for
+     * the user Nimbus and Supervisors use to authenticate with ZK.
+     */
+    @isString
+    public static final String STORM_ZOOKEEPER_SUPERACL = "storm.zookeeper.superACL";
+
+    /**
+     * The topology Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
+     */
+    @isString
+    public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME="storm.zookeeper.topology.auth.scheme";
+
+    /**
+     * The delegate for serializing metadata, should be used for serialized objects stored in zookeeper and on disk.
+     * This is NOT used for compressing serialized tuples sent between topologies.
+     */
+    @isString
+    public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate";
+
+    /**
+     * What blobstore implementation the storm client should use.
+     */
+    @isString
+    public static final String CLIENT_BLOBSTORE = "client.blobstore.class";
+
+
+    /**
+     * The blobstore super user has all read/write/admin permissions to all blobs - user running
+     * the blobstore.
+     */
+    @isString
+    public static final String BLOBSTORE_SUPERUSER = "blobstore.superuser";
+
+    /**
+     * What directory to use for the blobstore. The directory is expected to be an
+     * absolute path when using HDFS blobstore, for LocalFsBlobStore it could be either
+     * absolute or relative.
+     */
+    @isString
+    public static final String BLOBSTORE_DIR = "blobstore.dir";
+
+    /**
+     * Enable the blobstore cleaner. Certain blobstores may only want to run the cleaner
+     * on one daemon. Currently Nimbus handles setting this.
+     */
+    @isBoolean
+    public static final String BLOBSTORE_CLEANUP_ENABLE = "blobstore.cleanup.enable";
+
+    /**
+     * principal for nimbus/supervisor to use to access secure hdfs for the blobstore.
+     */
+    @isString
+    public static final String BLOBSTORE_HDFS_PRINCIPAL = "blobstore.hdfs.principal";
+
+    /**
+     * keytab for nimbus/supervisor to use to access secure hdfs for the blobstore.
+     */
+    @isString
+    public static final String BLOBSTORE_HDFS_KEYTAB = "blobstore.hdfs.keytab";
+
+    /**
+     *  Set replication factor for a blob in HDFS Blobstore Implementation
+     */
+    @isPositiveNumber
+    @isInteger
+    public static final String STORM_BLOBSTORE_REPLICATION_FACTOR = "storm.blobstore.replication.factor";
+
+    /**
+     * 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 don't have a DNS which supervisors/workers
+     * can utilize to find each other based on hostname got from calls to
+     * <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
+     */
+    @isString
+    public static final String STORM_LOCAL_HOSTNAME = "storm.local.hostname";
+
+    /**
+     * The host that the master server is running on, added only for backward compatibility,
+     * the usage deprecated in favor of nimbus.seeds config.
+     */
+    @Deprecated
+    @isString
+    public static final String NIMBUS_HOST = "nimbus.host";
+
+    /**
+     * List of seed nimbus hosts to use for leader nimbus discovery.
+     */
+    @isStringList
+    public static final String NIMBUS_SEEDS = "nimbus.seeds";
+
+    /**
+     * A list of users that are the only ones allowed to run user operation on storm cluster.
+     * To use this set nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
+     */
+    @isStringList
+    public static final String NIMBUS_USERS = "nimbus.users";
+
+    /**
+     * A list of groups , users belong to these groups are the only ones allowed to run user operation on storm cluster.
+     * To use this set nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
+     */
+    @isStringList
+    public static final String NIMBUS_GROUPS = "nimbus.groups";
+
+    /**
+     * The mode this Storm cluster is running in. Either "distributed" or "local".
+     */
+    @isString
+    public static final String STORM_CLUSTER_MODE = "storm.cluster.mode";
+
+    /**
+     * The root location at which Storm stores data in ZooKeeper.
+     */
+    @isString
+    public static final String STORM_ZOOKEEPER_ROOT = "storm.zookeeper.root";
+
+    /**
+     * A string representing the payload for topology Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication.
+     */
+    @isString
+    public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD="storm.zookeeper.topology.auth.payload";
+
+    /**
+     * The cluster Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
+     */
+    @isString
+    public static final String STORM_ZOOKEEPER_AUTH_SCHEME="storm.zookeeper.auth.scheme";
+
+    /**
+     * A string representing the payload for cluster Zookeeper authentication.
+     * It gets serialized using UTF-8 encoding during authentication.
+     * Note that if this is set to something with a secret (as when using
+     * digest authentication) then it should only be set in the
+     * storm-cluster-auth.yaml file.
+     * This file storm-cluster-auth.yaml should then be protected with
+     * appropriate permissions that deny access from workers.
+     */
+    @isString
+    public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD="storm.zookeeper.auth.payload";
+
+    /**
+     * What Network Topography detection classes should we use.
+     * Given a list of supervisor hostnames (or IP addresses), this class would return a list of
+     * rack names that correspond to the supervisors. This information is stored in Cluster.java, and
+     * is used in the resource aware scheduler.
+     */
+    @NotNull
+    @isImplementationOfClass(implementsClass = org.apache.storm.networktopography.DNSToSwitchMapping.class)
+    public static final String STORM_NETWORK_TOPOGRAPHY_PLUGIN = "storm.network.topography.plugin";
+
+    /**
+     * The jvm opts provided to workers launched by this supervisor for GC. All "%ID%" substrings are replaced
+     * with an identifier for this worker.  Because the JVM complains about multiple GC opts the topology
+     * can override this default value by setting topology.worker.gc.childopts.
+     */
+    @isStringOrStringList
+    public static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts";
+
+    /**
+     * The jvm opts provided to workers launched by this supervisor.
+     * All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%",
+     * "%WORKER-PORT%" and "%HEAP-MEM%" substrings are replaced with:
+     * %ID%          -> port (for backward compatibility),
+     * %WORKER-ID%   -> worker-id,
+     * %TOPOLOGY-ID%    -> topology-id,
+     * %WORKER-PORT% -> port.
+     * %HEAP-MEM% -> mem-onheap.
+     */
+    @isStringOrStringList
+    public static final String WORKER_CHILDOPTS = "worker.childopts";
+
+    /**
+     * The default heap memory size in MB per worker, used in the jvm -Xmx opts for launching the worker
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String WORKER_HEAP_MEMORY_MB = "worker.heap.memory.mb";
+
+    /**
+     * The total amount of memory (in MiB) a supervisor is allowed to give to its workers.
+     *  A default value will be set for this config if user does not override
+     */
+    @isPositiveNumber
+    public static final String SUPERVISOR_MEMORY_CAPACITY_MB = "supervisor.memory.capacity.mb";
+
+    /**
+     * The total amount of CPU resources a supervisor is allowed to give to its workers.
+     * By convention 1 cpu core should be about 100, but this can be adjusted if needed
+     * using 100 makes it simple to set the desired value to the capacity measurement
+     * for single threaded bolts.  A default value will be set for this config if user does not override
+     */
+    @isPositiveNumber
+    public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
+
+    /**
+     * 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.
+     */
+    @isBoolean
+    public static final String STORM_LOCAL_MODE_ZMQ = "storm.local.mode.zmq";
+
+    /**
+     * The transporter for communication among Storm tasks
+     */
+    @isString
+    public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport";
+
+    /**
+     * Netty based messaging: Is authentication required for Netty messaging from client worker process to server worker process.
+     */
+    @isBoolean
+    public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication";
+
+    /**
+     * Netty based messaging: The buffer size for send/recv buffer
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size";
+
+    /**
+     * Netty based messaging: Sets the backlog value to specify when the channel binds to a local address
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String STORM_MESSAGING_NETTY_SOCKET_BACKLOG = "storm.messaging.netty.socket.backlog";
+
+    /**
+     * Netty based messaging: The # of worker threads for the server.
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads";
+
+    /**
+     * If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of STORM_NETTY_MESSAGE_BATCH_SIZE bytes
+     */
+    @isInteger
+    public static final String STORM_NETTY_MESSAGE_BATCH_SIZE = "storm.messaging.netty.transfer.batch.size";
+
+    /**
+     * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible
+     *@deprecated "Since netty clients should never stop reconnecting - this does not make sense anymore.
+     */
+    @Deprecated
+    @isInteger
+    public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries";
+
+    /**
+     * Netty based messaging: The min # of milliseconds that a peer will wait.
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms";
+
+    /**
+     * Netty based messaging: The max # of milliseconds that a peer will wait.
+     */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
+    public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms";
+
+    /**
+     * Netty based messaging: The # of worker threads for the client.
+     */
+    @isInteger
+    public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads";
+
+    /**
+     * Should the supervior try to run the worker as the lauching user or not.  Defaults to false.
+     */
+    @isBoolean
+    public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user";
+
+    /**
+     * On some systems (windows for example) symlinks require special privileges that not everyone wants to
+     * grant a headless user.  You can completely disable the use of symlinks by setting this config to true, but
+     * by doing so you may also lose some features from storm.  For example the blobstore feature
+     * does not currently work without symlinks enabled.
+     */
+    @isBoolean
+    public static final String DISABLE_SYMLINKS = "storm.disable.symlinks";
+
+    /**
+     * The plugin that will convert a principal to a local user.
+     */
+    @isString
+    public static final String STORM_PRINCIPAL_TO_LOCAL_PLUGIN = "storm.principal.tolocal";
+
+    /**
+     * The plugin that will provide user groups service
+     */
+    @isString
+    public static final String STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN = "storm.group.mapping.service";
+
+    /**
+     * A list of credential renewers that nimbus should load.
+     */
+    @isStringList
+    public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes";
+
+    /**
+     * A list of plugins that nimbus should load during submit topology to populate
+     * credentials on user's behalf.
+     */
+    @isStringList
+    public static final String NIMBUS_AUTO_CRED_PLUGINS = "nimbus.autocredential.plugins.classes";
+
+    /**
+     * Class name of the HTTP credentials plugin for the UI.
+     */
+    @isString
+    public static final String UI_HTTP_CREDS_PLUGIN = "ui.http.creds.plugin";
+
+    /**
+     * Class name of the HTTP credentials plugin for DRPC.
+     */
+    @isString
+    public static final String DRPC_HTTP_CREDS_PLUGIN = "drpc.http.creds.plugin";
+
+    /**
+     * A list of users that run the supervisors and should be authorized to interact with
+     * nimbus as a supervisor would.  To use this set
+     * nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
+     */
+    @isStringList
+    public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users";
+
+    /**
+     * A list of users that are cluster admins and can run any command.  To use this set
+     * nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
+     */
+    @isStringList
+    public static final String NIMBUS_ADMINS = "nimbus.admins";
+
+    /**
+     *  For secure mode we would want to turn on this config
+     *  By default this is turned off assuming the default is insecure
+     */
+    @isBoolean
+    public static final String STORM_BLOBSTORE_ACL_VALIDATION_ENABLED = "storm.blobstore.acl.validation.enabled";
+
+    /**
+     * What buffer size to use for the blobstore uploads.
+     */
+    @isPositiveNumber
+    @isInteger
+    public static final String STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES = "storm.blobstore.inputstream.buffer.size.bytes";
+
+    /**
+     * FQCN of a class that implements {@code ISubmitterHook} @see ISubmitterHook for details.
+     */
+    @isString
+    public static final String STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN = "storm.topology.submission.notifier.plugin.class";
+
+    /**
+     * Impersonation user ACL config entries.
+     */
+    @isMapEntryCustom(keyValidatorClasses = {ConfigValidation.StringValidator.class}, valueValidatorClasses = {ConfigValidation.ImpersonationAclUserEntryValidator.class})
+    public static final String NIMBUS_IMPERSONATION_ACL = "nimbus.impersonation.acl";
+
+    /**
+     * Full path to the worker-laucher executable that will be used to lauch workers when
+     * SUPERVISOR_RUN_WORKER_AS_USER is set to true.
+     */
+    @isString
+    public static final String SUPERVISOR_WORKER_LAUNCHER = "supervisor.worker.launcher";
+
+    /**
+     * A directory on the local filesystem used by Storm for any local
+     * filesystem usage it needs. The directory must exist and the Storm daemons must
+     * have permission to read/write from this location.
+     */
+    @isString
+    public static final String STORM_LOCAL_DIR = "storm.local.dir";
+
+    /**
+     * The workers-artifacts directory (where we place all workers' logs), can be either absolute or relative.
+     * By default, ${storm.log.dir}/workers-artifacts is where worker logs go.
+     * If the setting is a relative directory, it is relative to storm.log.dir.
+     */
+    @isString
+    public static final String STORM_WORKERS_ARTIFACTS_DIR = "storm.workers.artifacts.dir";
+
+    /**
+     * A list of hosts of Exhibitor servers used to discover/maintain connection to ZooKeeper cluster.
+     * Any configured ZooKeeper servers will be used for the curator/exhibitor backup connection string.
+     */
+    @isStringList
+    public static final String STORM_EXHIBITOR_SERVERS = "storm.exhibitor.servers";
+
+    /**
+     * The port Storm will use to connect to each of the exhibitor servers.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String STORM_EXHIBITOR_PORT = "storm.exhibitor.port";
+
+    /*
+ * How often to poll Exhibitor cluster in millis.
+ */
+    @isString
+    public static final String STORM_EXHIBITOR_URIPATH="storm.exhibitor.poll.uripath";
+
+    /**
+     * How often to poll Exhibitor cluster in millis.
+     */
+    @isInteger
+    public static final String STORM_EXHIBITOR_POLL="storm.exhibitor.poll.millis";
+
+    /**
+     * The number of times to retry an Exhibitor operation.
+     */
+    @isInteger
+    public static final String STORM_EXHIBITOR_RETRY_TIMES="storm.exhibitor.retry.times";
+
+    /**
+     * The interval between retries of an Exhibitor operation.
+     */
+    @isInteger
+    public static final String STORM_EXHIBITOR_RETRY_INTERVAL="storm.exhibitor.retry.interval";
+
+    /**
+     * The ceiling of the interval between retries of an Exhibitor operation.
+     */
+    @isInteger
+    public static final String STORM_EXHIBITOR_RETRY_INTERVAL_CEILING="storm.exhibitor.retry.intervalceiling.millis";
+
+    /**
+     * The connection timeout for clients to ZooKeeper.
+     */
+    @isInteger
+    public static final String STORM_ZOOKEEPER_CONNECTION_TIMEOUT = "storm.zookeeper.connection.timeout";
+
+    /**
+     * The session timeout for clients to ZooKeeper.
+     */
+    @isInteger
+    public static final String STORM_ZOOKEEPER_SESSION_TIMEOUT = "storm.zookeeper.session.timeout";
+
+    /**
+     * The interval between retries of a Zookeeper operation.
+     */
+    @isInteger
+    public static final String STORM_ZOOKEEPER_RETRY_INTERVAL="storm.zookeeper.retry.interval";
+
+    /**
+     * The ceiling of the interval between retries of a Zookeeper operation.
+     */
+    @isInteger
+    public static final String STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING="storm.zookeeper.retry.intervalceiling.millis";
+
+    /**
+     * The number of times to retry a Zookeeper operation.
+     */
+    @isInteger
+    public static final String STORM_ZOOKEEPER_RETRY_TIMES="storm.zookeeper.retry.times";
+
+    /**
+     * The ClusterState factory that worker will use to create a ClusterState
+     * to store state in. Defaults to ZooKeeper.
+     */
+    @isString
+    public static final String STORM_CLUSTER_STATE_STORE = "storm.cluster.state.store";
+
+    /**
+     * How often this worker should heartbeat to the supervisor.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs";
+
+    /**
+     * How often a task should heartbeat its status to the master.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs";
+
+    /**
+     * How often a task should sync its connections with other tasks (if a task is
+     * reassigned, the other tasks sending messages to it need to refresh their connections).
+     * In general though, when a reassignment happens other tasks will be notified
+     * almost immediately. This configuration is here just in case that notification doesn't
+     * come through.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TASK_REFRESH_POLL_SECS = "task.refresh.poll.secs";
+
+    /**
+     * The Access Control List for the DRPC Authorizer.
+     * @see org.apache.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
+     */
+    @isType(type=Map.class)
+    public static final String DRPC_AUTHORIZER_ACL = "drpc.authorizer.acl";
+
+    /**
+     * File name of the DRPC Authorizer ACL.
+     * @see org.apache.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
+     */
+    @isString
+    public static final String DRPC_AUTHORIZER_ACL_FILENAME = "drpc.authorizer.acl.filename";
+
+    /**
+     * Whether the DRPCSimpleAclAuthorizer should deny requests for operations
+     * involving functions that have no explicit ACL entry. When set to false
+     * (the default) DRPC functions that have no entry in the ACL will be
+     * permitted, which is appropriate for a development environment. When set
+     * to true, explicit ACL entries are required for every DRPC function, and
+     * any request for functions will be denied.
+     * @see org.apache.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
+     */
+    @isBoolean
+    public static final String DRPC_AUTHORIZER_ACL_STRICT = "drpc.authorizer.acl.strict";
+
+    /**
+     * root directory of the storm cgroup hierarchy
+     */
+    @isString
+    public static final String STORM_CGROUP_HIERARCHY_DIR = "storm.cgroup.hierarchy.dir";
+
+    /**
+     * The number of Buckets
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String NUM_STAT_BUCKETS = "num.stat.buckets";
+
+    public static void setClasspath(Map conf, String cp) {
+        conf.put(Config.TOPOLOGY_CLASSPATH, cp);
+    }
+
+    public void setClasspath(String cp) {
+        setClasspath(this, cp);
+    }
+
+    public static void setEnvironment(Map conf, Map env) {
+        conf.put(Config.TOPOLOGY_ENVIRONMENT, env);
+    }
+
+    public void setEnvironment(Map env) {
+        setEnvironment(this, env);
+    }
+
+    public static void setDebug(Map conf, boolean isOn) {
+        conf.put(Config.TOPOLOGY_DEBUG, isOn);
+    }
+
+    public void setDebug(boolean isOn) {
+        setDebug(this, isOn);
+    }
+
+    public static void setNumWorkers(Map conf, int workers) {
+        conf.put(Config.TOPOLOGY_WORKERS, workers);
+    }
+
+    public void setNumWorkers(int workers) {
+        setNumWorkers(this, workers);
+    }
+
+    public static void setNumAckers(Map conf, int numExecutors) {
+        conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, numExecutors);
+    }
+
+    public void setNumAckers(int numExecutors) {
+        setNumAckers(this, numExecutors);
+    }
+
+    public static void setNumEventLoggers(Map conf, int numExecutors) {
+        conf.put(Config.TOPOLOGY_EVENTLOGGER_EXECUTORS, numExecutors);
+    }
+
+    public void setNumEventLoggers(int numExecutors) {
+        setNumEventLoggers(this, numExecutors);
+    }
+
+
+    public static void setMessageTimeoutSecs(Map conf, int secs) {
+        conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs);
+    }
+
+    public void setMessageTimeoutSecs(int secs) {
+        setMessageTimeoutSecs(this, secs);
+    }
+
+    public static void registerSerialization(Map conf, Class klass) {
+        getRegisteredSerializations(conf).add(klass.getName());
+    }
+
+    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);
+    }
+
+    public void registerSerialization(Class klass, Class<? extends Serializer> serializerClass) {
+        registerSerialization(this, klass, serializerClass);
+    }
+
+    public static void registerMetricsConsumer(Map conf, Class klass, Object argument, long parallelismHint) {
+        HashMap m = new HashMap();
+        m.put("class", klass.getCanonicalName());
+        m.put("parallelism.hint", parallelismHint);
+        m.put("argument", argument);
+
+        List l = (List)conf.get(TOPOLOGY_METRICS_CONSUMER_REGISTER);
+        if (l == null) { l = new ArrayList(); }
+        l.add(m);
+        conf.put(TOPOLOGY_METRICS_CONSUMER_REGISTER, l);
+    }
+
+    public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) {
+        registerMetricsConsumer(this, klass, argument, parallelismHint);
+    }
+
+    public static void registerMetricsConsumer(Map conf, Class klass, long parallelismHint) {
+        registerMetricsConsumer(conf, klass, null, parallelismHint);
+    }
+
+    public void registerMetricsConsumer(Class klass, long parallelismHint) {
+        registerMetricsConsumer(this, klass, parallelismHint);
+    }
+
+    public static void registerMetricsConsumer(Map conf, Class klass) {
+        registerMetricsConsumer(conf, klass, null, 1L);
+    }
+
+    public void registerMetricsConsumer(Class klass) {
+        registerMetricsConsumer(this, klass);
+    }
+
+    public static void registerDecorator(Map conf, Class<? extends IKryoDecorator> klass) {
+        getRegisteredDecorators(conf).add(klass.getName());
+    }
+
+    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());
+    }
+
+    public void setKryoFactory(Class<? extends IKryoFactory> klass) {
+        setKryoFactory(this, klass);
+    }
+
+    public static void setSkipMissingKryoRegistrations(Map conf, boolean skip) {
+        conf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, skip);
+    }
+
+    public void setSkipMissingKryoRegistrations(boolean skip) {
+        setSkipMissingKryoRegistrations(this, skip);
+    }
+
+    public static void setMaxTaskParallelism(Map conf, int max) {
+        conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max);
+    }
+
+    public void setMaxTaskParallelism(int max) {
+        setMaxTaskParallelism(this, max);
+    }
+
+    public static void setMaxSpoutPending(Map conf, int max) {
+        conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max);
+    }
+
+    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)) {
+            ret = new ArrayList();
+        } else {
+            ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_REGISTER));
+        }
+        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));
+        }
+        conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret);
+        return ret;
+    }
+
+    /**
+     * set the max heap size allow per worker for this topology
+     * @param size
+     */
+    public void setTopologyWorkerMaxHeapSize(Number size) {
+        if(size != null) {
+            this.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, size);
+        }
+    }
+
+    /**
+     * set the priority for a topology
+     * @param priority
+     */
+    public void setTopologyPriority(int priority) {
+        this.put(Config.TOPOLOGY_PRIORITY, priority);
+    }
+
+    /**
+     * Takes as input the strategy class name. Strategy must implement the IStrategy interface
+     * @param clazz class of the strategy to use
+     */
+    public void setTopologyStrategy(Class<? extends IStrategy> clazz) {
+        if (clazz != null) {
+            this.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, clazz.getName());
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/Constants.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/Constants.java b/storm-client/src/jvm/org/apache/storm/Constants.java
new file mode 100644
index 0000000..b86b1ca
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/Constants.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import org.apache.storm.coordination.CoordinatedBolt;
+
+import java.util.Arrays;
+import java.util.List;
+
+
+public class Constants {
+    public static final String COORDINATED_STREAM_ID = CoordinatedBolt.class.getName() + "/coord-stream";
+
+    public static final long SYSTEM_TASK_ID = -1;
+    public static final List<Long> SYSTEM_EXECUTOR_ID = Arrays.asList(-1L, -1L);
+    public static final String SYSTEM_COMPONENT_ID = "__system";
+    public static final String SYSTEM_TICK_STREAM_ID = "__tick";
+    public static final String METRICS_COMPONENT_ID_PREFIX = "__metrics";
+    public static final String METRICS_STREAM_ID = "__metrics";
+    public static final String METRICS_TICK_STREAM_ID = "__metrics_tick";
+    public static final String CREDENTIALS_CHANGED_STREAM_ID = "__credentials";
+
+    public static final Object TOPOLOGY = "topology";
+    public static final String SYSTEM_TOPOLOGY = "system-topology";
+    public static final String STORM_CONF = "storm-conf";
+    public static final String STORM_ID = "storm-id";
+    public static final String WORKER_ID = "worker-id";
+    public static final String CONF = "conf";
+    public static final String PORT = "port";
+    public static final String TASK_TO_COMPONENT = "task->component";
+    public static final String COMPONENT_TO_SORTED_TASKS = "component->sorted-tasks";
+    public static final String COMPONENT_TO_STREAM_TO_FIELDS = "component->stream->fields";
+    public static final String TASK_IDS = "task-ids";
+    public static final String DEFAULT_SHARED_RESOURCES = "default-shared-resources";
+    public static final String USER_SHARED_RESOURCES = "user-shared-resources";
+    public static final String USER_TIMER = "user-timer";
+    public static final String TRANSFER_FN = "transfer-fn";
+    public static final String SUICIDE_FN = "suicide-fn";
+    public static final String THROTTLE_ON = "throttle-on";
+    public static final String EXECUTOR_RECEIVE_QUEUE_MAP = "executor-receive-queue-map";
+    public static final String STORM_ACTIVE_ATOM = "storm-active-atom";
+    public static final String COMPONENT_TO_DEBUG_ATOM = "storm-component->debug-atom";
+    public static final Object LOAD_MAPPING = "load-mapping";
+}
+    

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/ICredentialsListener.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/ICredentialsListener.java b/storm-client/src/jvm/org/apache/storm/ICredentialsListener.java
new file mode 100644
index 0000000..5eed024
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/ICredentialsListener.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm;
+
+import java.util.Map;
+
+/**
+ * Allows a bolt or a spout to be informed when the credentials of the topology have changed.
+ */
+public interface ICredentialsListener {
+    /**
+     * Called when the credentials of a topology have changed.
+     * @param credentials the new credentials, could be null.
+     */
+    public void setCredentials(Map<String,String> credentials);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/ILocalCluster.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/ILocalCluster.java b/storm-client/src/jvm/org/apache/storm/ILocalCluster.java
new file mode 100644
index 0000000..7266b73
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/ILocalCluster.java
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import java.util.Map;
+
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.Credentials;
+import org.apache.storm.generated.KillOptions;
+import org.apache.storm.generated.RebalanceOptions;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.generated.TopologyInfo;
+import org.apache.thrift.TException;
+
+/**
+ * This is here mostly for backwards compatibility.
+ */
+public interface ILocalCluster extends AutoCloseable {
+    /**
+     * Close this class to kill the topology.
+     * This is here mostly for backwards compatibility.
+     */
+    interface ILocalTopology extends AutoCloseable {
+
+    }
+    
+    /**
+     * Submit a topology to be run in local mode
+     * @param topologyName the name of the topology to use
+     * @param conf the config for the topology
+     * @param topology the topology itself.
+     * @return an AutoCloseable that will kill the topology.
+     * @throws TException on any error from nimbus
+     */
+    ILocalTopology submitTopology(String topologyName, Map<String, Object> conf, StormTopology topology) throws TException;
+    
+    /**
+     * Submit a topology to be run in local mode
+     * @param topologyName the name of the topology to use
+     * @param conf the config for the topology
+     * @param topology the topology itself.
+     * @param submitOpts options for topology
+     * @return an AutoCloseable that will kill the topology.
+     * @throws TException on any error from nimbus
+     */
+    ILocalTopology submitTopologyWithOpts(String topologyName, Map<String, Object> conf, StormTopology topology, SubmitOptions submitOpts) throws TException;
+    
+    /**
+     * Upload new credentials to a topology.
+     * @param topologyName the name of the topology
+     * @param creds the new credentails for the topology.
+     * @throws TException on any error from nimbus
+     */
+    void uploadNewCredentials(String topologyName, Credentials creds) throws TException;
+    
+    /**
+     * Kill a topology (if you are not using ILocalTopology)
+     * @param topologyName the name of the topology
+     * @throws TException on any error from nimbus
+     */
+    void killTopology(String topologyName) throws TException;
+    
+    /**
+     * Kill a topology (if you are not using ILocalTopology)
+     * @param topologyName the name of the topology
+     * @param options for how to kill the topology
+     * @throws TException on any error from nimbus
+     */
+    void killTopologyWithOpts(String name, KillOptions options) throws TException;
+    
+    /**
+     * Activate a topology
+     * @param topologyName the name of the topology to activate
+     * @throws TException on any error from nimbus
+     */
+    void activate(String topologyName) throws TException;
+    
+    /**
+     * Deactivate a topology
+     * @param topologyName the name of the topology to deactivate
+     * @throws TException on any error from nimbus
+     */
+    void deactivate(String topologyName) throws TException;
+    
+    /**
+     * Rebalance a topology
+     * @param name the name of the topology
+     * @param options options for rebalanceing the topology.
+     * @throws TException on any error from nimbus
+     */
+    void rebalance(String name, RebalanceOptions options) throws TException;
+    
+    /**
+     * Shut down the cluster.
+     * @deprecated use {@link #close()} instead.
+     */
+    @Deprecated
+    void shutdown();
+    
+    /**
+     * The config of a topology as a JSON string
+     * @param id the id of the topology (not the name)
+     * @return The config of a topology as a JSON string
+     * @throws TException on any error from nimbus
+     */
+    String getTopologyConf(String id) throws TException;
+    
+    /**
+     * Get the compiled storm topology 
+     * @param id the id of the topology (not the name)
+     * @return the compiled storm topology
+     * @throws TException on any error from nimbus
+     */
+    StormTopology getTopology(String id) throws TException;
+    
+    /**
+     * @return a summary of the current state of the cluster
+     * @throws TException on any error from nimbus
+     */
+    ClusterSummary getClusterInfo() throws TException;
+    
+    /**
+     * Get the state of a topology
+     * @param id the id of the topology (not the name)
+     * @return the state of a topology
+     * @throws TException on any error from nimbus
+     */
+    TopologyInfo getTopologyInfo(String id) throws TException;
+    
+    /**
+     * This is intended for internal testing only.
+     * @return an internal class that holds the state of the cluster.
+     */
+    IStormClusterState getClusterState();
+    
+    /**
+     * Advance the cluster time when the cluster is using SimulatedTime.
+     * This is intended for internal testing only.
+     * @param secs the number of seconds to advance time
+     * @throws InterruptedException
+     */
+    void advanceClusterTime(int secs) throws InterruptedException;
+    
+    /**
+     * Advance the cluster time when the cluster is using SimulatedTime.
+     * This is intended for internal testing only.
+     * @param secs the number of seconds to advance time
+     * @param steps the number of steps we should take when advancing simulated time
+     * @throws InterruptedException
+     */
+    void advanceClusterTime(int secs, int step) throws InterruptedException;
+    
+    /**
+     * If the cluster is tracked get the id for the tracked cluster.
+     * This is intended for internal testing only.
+     * @return the id of the tracked cluster
+     */
+    String getTrackedId();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java b/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java
new file mode 100644
index 0000000..68b60d6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.generated.DistributedRPC;
+import org.apache.storm.generated.DistributedRPCInvocations;
+
+
+public interface ILocalDRPC extends DistributedRPC.Iface, DistributedRPCInvocations.Iface, Shutdownable, AutoCloseable {
+    /**
+     * Get the ID of the service.  This is used internally if multiple local DRPC clusters are in use at one time.
+     */
+    public String getServiceId();
+    
+    /**
+     * @deprecated use {@link #close()} instead
+     */
+    @Deprecated
+    public void shutdown();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/ISubmitterHook.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/ISubmitterHook.java b/storm-client/src/jvm/org/apache/storm/ISubmitterHook.java
new file mode 100644
index 0000000..ac80788
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/ISubmitterHook.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.TopologyInfo;
+
+import java.util.Map;
+
+/**
+ * if FQCN of an implementation of this class is specified by setting the config storm.topology.submission.notifier.plugin.class,
+ * that class's notify method will be invoked when a topology is successfully submitted via StormSubmitter class.
+ */
+public interface ISubmitterHook {
+    public void notify(TopologyInfo topologyInfo, Map stormConf, StormTopology topology) throws IllegalAccessException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/LogWriter.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/LogWriter.java b/storm-client/src/jvm/org/apache/storm/LogWriter.java
new file mode 100644
index 0000000..b9bcf80
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/LogWriter.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.io.InputStream;
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Launch a sub process and write files out to logs.
+ */
+public class LogWriter extends Thread {
+    private Logger logger;
+    private BufferedReader in;
+
+    public LogWriter(InputStream in, Logger logger) {
+        this.in = new BufferedReader(new InputStreamReader(in));
+        this.logger = logger;
+    }
+
+    public void run() {
+        Logger logger = this.logger;
+        BufferedReader in = this.in;
+        String line;
+        try {
+            while ((line = in.readLine()) != null) {
+                logger.info(line);
+            }
+        } catch (IOException e) {
+            logger.error("Internal ERROR", e);
+        } finally {
+            try {
+                in.close();
+            } catch (IOException e) {
+                logger.error("Internal ERROR", e);
+            }
+        }
+    }
+
+    public void close() throws Exception {
+        this.join();
+    }
+
+    public static void main(String [] args) throws Exception {
+        ProcessBuilder pb = new ProcessBuilder(args);
+        Process p = pb.start();
+        LogWriter err = null;
+        LogWriter in = null;
+        int ret = -1;
+        try {
+            Logger logger = LoggerFactory.getLogger("STDERR");
+            err = new LogWriter(p.getErrorStream(), logger);
+            err.start();
+            in = new LogWriter(p.getInputStream(), logger);
+            in.start();
+            ret = p.waitFor();
+        } finally {
+          if (err != null) err.close();
+          if (in != null) in.close();
+        }
+        System.exit(ret);
+    }
+}


[13/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/StormTopology.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/StormTopology.java b/storm-client/src/jvm/org/apache/storm/generated/StormTopology.java
new file mode 100644
index 0000000..caec6c6
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/StormTopology.java
@@ -0,0 +1,1272 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class StormTopology implements org.apache.thrift.TBase<StormTopology, StormTopology._Fields>, java.io.Serializable, Cloneable, Comparable<StormTopology> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormTopology");
+
+  private static final org.apache.thrift.protocol.TField SPOUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("spouts", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField BOLTS_FIELD_DESC = new org.apache.thrift.protocol.TField("bolts", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField STATE_SPOUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("state_spouts", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField WORKER_HOOKS_FIELD_DESC = new org.apache.thrift.protocol.TField("worker_hooks", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField DEPENDENCY_JARS_FIELD_DESC = new org.apache.thrift.protocol.TField("dependency_jars", org.apache.thrift.protocol.TType.LIST, (short)5);
+  private static final org.apache.thrift.protocol.TField DEPENDENCY_ARTIFACTS_FIELD_DESC = new org.apache.thrift.protocol.TField("dependency_artifacts", org.apache.thrift.protocol.TType.LIST, (short)6);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new StormTopologyStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new StormTopologyTupleSchemeFactory());
+  }
+
+  private Map<String,SpoutSpec> spouts; // required
+  private Map<String,Bolt> bolts; // required
+  private Map<String,StateSpoutSpec> state_spouts; // required
+  private List<ByteBuffer> worker_hooks; // optional
+  private List<String> dependency_jars; // optional
+  private List<String> dependency_artifacts; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SPOUTS((short)1, "spouts"),
+    BOLTS((short)2, "bolts"),
+    STATE_SPOUTS((short)3, "state_spouts"),
+    WORKER_HOOKS((short)4, "worker_hooks"),
+    DEPENDENCY_JARS((short)5, "dependency_jars"),
+    DEPENDENCY_ARTIFACTS((short)6, "dependency_artifacts");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SPOUTS
+          return SPOUTS;
+        case 2: // BOLTS
+          return BOLTS;
+        case 3: // STATE_SPOUTS
+          return STATE_SPOUTS;
+        case 4: // WORKER_HOOKS
+          return WORKER_HOOKS;
+        case 5: // DEPENDENCY_JARS
+          return DEPENDENCY_JARS;
+        case 6: // DEPENDENCY_ARTIFACTS
+          return DEPENDENCY_ARTIFACTS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.WORKER_HOOKS,_Fields.DEPENDENCY_JARS,_Fields.DEPENDENCY_ARTIFACTS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SPOUTS, new org.apache.thrift.meta_data.FieldMetaData("spouts", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpoutSpec.class))));
+    tmpMap.put(_Fields.BOLTS, new org.apache.thrift.meta_data.FieldMetaData("bolts", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Bolt.class))));
+    tmpMap.put(_Fields.STATE_SPOUTS, new org.apache.thrift.meta_data.FieldMetaData("state_spouts", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StateSpoutSpec.class))));
+    tmpMap.put(_Fields.WORKER_HOOKS, new org.apache.thrift.meta_data.FieldMetaData("worker_hooks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
+    tmpMap.put(_Fields.DEPENDENCY_JARS, new org.apache.thrift.meta_data.FieldMetaData("dependency_jars", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.DEPENDENCY_ARTIFACTS, new org.apache.thrift.meta_data.FieldMetaData("dependency_artifacts", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormTopology.class, metaDataMap);
+  }
+
+  public StormTopology() {
+  }
+
+  public StormTopology(
+    Map<String,SpoutSpec> spouts,
+    Map<String,Bolt> bolts,
+    Map<String,StateSpoutSpec> state_spouts)
+  {
+    this();
+    this.spouts = spouts;
+    this.bolts = bolts;
+    this.state_spouts = state_spouts;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StormTopology(StormTopology other) {
+    if (other.is_set_spouts()) {
+      Map<String,SpoutSpec> __this__spouts = new HashMap<String,SpoutSpec>(other.spouts.size());
+      for (Map.Entry<String, SpoutSpec> other_element : other.spouts.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        SpoutSpec other_element_value = other_element.getValue();
+
+        String __this__spouts_copy_key = other_element_key;
+
+        SpoutSpec __this__spouts_copy_value = new SpoutSpec(other_element_value);
+
+        __this__spouts.put(__this__spouts_copy_key, __this__spouts_copy_value);
+      }
+      this.spouts = __this__spouts;
+    }
+    if (other.is_set_bolts()) {
+      Map<String,Bolt> __this__bolts = new HashMap<String,Bolt>(other.bolts.size());
+      for (Map.Entry<String, Bolt> other_element : other.bolts.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Bolt other_element_value = other_element.getValue();
+
+        String __this__bolts_copy_key = other_element_key;
+
+        Bolt __this__bolts_copy_value = new Bolt(other_element_value);
+
+        __this__bolts.put(__this__bolts_copy_key, __this__bolts_copy_value);
+      }
+      this.bolts = __this__bolts;
+    }
+    if (other.is_set_state_spouts()) {
+      Map<String,StateSpoutSpec> __this__state_spouts = new HashMap<String,StateSpoutSpec>(other.state_spouts.size());
+      for (Map.Entry<String, StateSpoutSpec> other_element : other.state_spouts.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        StateSpoutSpec other_element_value = other_element.getValue();
+
+        String __this__state_spouts_copy_key = other_element_key;
+
+        StateSpoutSpec __this__state_spouts_copy_value = new StateSpoutSpec(other_element_value);
+
+        __this__state_spouts.put(__this__state_spouts_copy_key, __this__state_spouts_copy_value);
+      }
+      this.state_spouts = __this__state_spouts;
+    }
+    if (other.is_set_worker_hooks()) {
+      List<ByteBuffer> __this__worker_hooks = new ArrayList<ByteBuffer>(other.worker_hooks);
+      this.worker_hooks = __this__worker_hooks;
+    }
+    if (other.is_set_dependency_jars()) {
+      List<String> __this__dependency_jars = new ArrayList<String>(other.dependency_jars);
+      this.dependency_jars = __this__dependency_jars;
+    }
+    if (other.is_set_dependency_artifacts()) {
+      List<String> __this__dependency_artifacts = new ArrayList<String>(other.dependency_artifacts);
+      this.dependency_artifacts = __this__dependency_artifacts;
+    }
+  }
+
+  public StormTopology deepCopy() {
+    return new StormTopology(this);
+  }
+
+  @Override
+  public void clear() {
+    this.spouts = null;
+    this.bolts = null;
+    this.state_spouts = null;
+    this.worker_hooks = null;
+    this.dependency_jars = null;
+    this.dependency_artifacts = null;
+  }
+
+  public int get_spouts_size() {
+    return (this.spouts == null) ? 0 : this.spouts.size();
+  }
+
+  public void put_to_spouts(String key, SpoutSpec val) {
+    if (this.spouts == null) {
+      this.spouts = new HashMap<String,SpoutSpec>();
+    }
+    this.spouts.put(key, val);
+  }
+
+  public Map<String,SpoutSpec> get_spouts() {
+    return this.spouts;
+  }
+
+  public void set_spouts(Map<String,SpoutSpec> spouts) {
+    this.spouts = spouts;
+  }
+
+  public void unset_spouts() {
+    this.spouts = null;
+  }
+
+  /** Returns true if field spouts is set (has been assigned a value) and false otherwise */
+  public boolean is_set_spouts() {
+    return this.spouts != null;
+  }
+
+  public void set_spouts_isSet(boolean value) {
+    if (!value) {
+      this.spouts = null;
+    }
+  }
+
+  public int get_bolts_size() {
+    return (this.bolts == null) ? 0 : this.bolts.size();
+  }
+
+  public void put_to_bolts(String key, Bolt val) {
+    if (this.bolts == null) {
+      this.bolts = new HashMap<String,Bolt>();
+    }
+    this.bolts.put(key, val);
+  }
+
+  public Map<String,Bolt> get_bolts() {
+    return this.bolts;
+  }
+
+  public void set_bolts(Map<String,Bolt> bolts) {
+    this.bolts = bolts;
+  }
+
+  public void unset_bolts() {
+    this.bolts = null;
+  }
+
+  /** Returns true if field bolts is set (has been assigned a value) and false otherwise */
+  public boolean is_set_bolts() {
+    return this.bolts != null;
+  }
+
+  public void set_bolts_isSet(boolean value) {
+    if (!value) {
+      this.bolts = null;
+    }
+  }
+
+  public int get_state_spouts_size() {
+    return (this.state_spouts == null) ? 0 : this.state_spouts.size();
+  }
+
+  public void put_to_state_spouts(String key, StateSpoutSpec val) {
+    if (this.state_spouts == null) {
+      this.state_spouts = new HashMap<String,StateSpoutSpec>();
+    }
+    this.state_spouts.put(key, val);
+  }
+
+  public Map<String,StateSpoutSpec> get_state_spouts() {
+    return this.state_spouts;
+  }
+
+  public void set_state_spouts(Map<String,StateSpoutSpec> state_spouts) {
+    this.state_spouts = state_spouts;
+  }
+
+  public void unset_state_spouts() {
+    this.state_spouts = null;
+  }
+
+  /** Returns true if field state_spouts is set (has been assigned a value) and false otherwise */
+  public boolean is_set_state_spouts() {
+    return this.state_spouts != null;
+  }
+
+  public void set_state_spouts_isSet(boolean value) {
+    if (!value) {
+      this.state_spouts = null;
+    }
+  }
+
+  public int get_worker_hooks_size() {
+    return (this.worker_hooks == null) ? 0 : this.worker_hooks.size();
+  }
+
+  public java.util.Iterator<ByteBuffer> get_worker_hooks_iterator() {
+    return (this.worker_hooks == null) ? null : this.worker_hooks.iterator();
+  }
+
+  public void add_to_worker_hooks(ByteBuffer elem) {
+    if (this.worker_hooks == null) {
+      this.worker_hooks = new ArrayList<ByteBuffer>();
+    }
+    this.worker_hooks.add(elem);
+  }
+
+  public List<ByteBuffer> get_worker_hooks() {
+    return this.worker_hooks;
+  }
+
+  public void set_worker_hooks(List<ByteBuffer> worker_hooks) {
+    this.worker_hooks = worker_hooks;
+  }
+
+  public void unset_worker_hooks() {
+    this.worker_hooks = null;
+  }
+
+  /** Returns true if field worker_hooks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_worker_hooks() {
+    return this.worker_hooks != null;
+  }
+
+  public void set_worker_hooks_isSet(boolean value) {
+    if (!value) {
+      this.worker_hooks = null;
+    }
+  }
+
+  public int get_dependency_jars_size() {
+    return (this.dependency_jars == null) ? 0 : this.dependency_jars.size();
+  }
+
+  public java.util.Iterator<String> get_dependency_jars_iterator() {
+    return (this.dependency_jars == null) ? null : this.dependency_jars.iterator();
+  }
+
+  public void add_to_dependency_jars(String elem) {
+    if (this.dependency_jars == null) {
+      this.dependency_jars = new ArrayList<String>();
+    }
+    this.dependency_jars.add(elem);
+  }
+
+  public List<String> get_dependency_jars() {
+    return this.dependency_jars;
+  }
+
+  public void set_dependency_jars(List<String> dependency_jars) {
+    this.dependency_jars = dependency_jars;
+  }
+
+  public void unset_dependency_jars() {
+    this.dependency_jars = null;
+  }
+
+  /** Returns true if field dependency_jars is set (has been assigned a value) and false otherwise */
+  public boolean is_set_dependency_jars() {
+    return this.dependency_jars != null;
+  }
+
+  public void set_dependency_jars_isSet(boolean value) {
+    if (!value) {
+      this.dependency_jars = null;
+    }
+  }
+
+  public int get_dependency_artifacts_size() {
+    return (this.dependency_artifacts == null) ? 0 : this.dependency_artifacts.size();
+  }
+
+  public java.util.Iterator<String> get_dependency_artifacts_iterator() {
+    return (this.dependency_artifacts == null) ? null : this.dependency_artifacts.iterator();
+  }
+
+  public void add_to_dependency_artifacts(String elem) {
+    if (this.dependency_artifacts == null) {
+      this.dependency_artifacts = new ArrayList<String>();
+    }
+    this.dependency_artifacts.add(elem);
+  }
+
+  public List<String> get_dependency_artifacts() {
+    return this.dependency_artifacts;
+  }
+
+  public void set_dependency_artifacts(List<String> dependency_artifacts) {
+    this.dependency_artifacts = dependency_artifacts;
+  }
+
+  public void unset_dependency_artifacts() {
+    this.dependency_artifacts = null;
+  }
+
+  /** Returns true if field dependency_artifacts is set (has been assigned a value) and false otherwise */
+  public boolean is_set_dependency_artifacts() {
+    return this.dependency_artifacts != null;
+  }
+
+  public void set_dependency_artifacts_isSet(boolean value) {
+    if (!value) {
+      this.dependency_artifacts = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SPOUTS:
+      if (value == null) {
+        unset_spouts();
+      } else {
+        set_spouts((Map<String,SpoutSpec>)value);
+      }
+      break;
+
+    case BOLTS:
+      if (value == null) {
+        unset_bolts();
+      } else {
+        set_bolts((Map<String,Bolt>)value);
+      }
+      break;
+
+    case STATE_SPOUTS:
+      if (value == null) {
+        unset_state_spouts();
+      } else {
+        set_state_spouts((Map<String,StateSpoutSpec>)value);
+      }
+      break;
+
+    case WORKER_HOOKS:
+      if (value == null) {
+        unset_worker_hooks();
+      } else {
+        set_worker_hooks((List<ByteBuffer>)value);
+      }
+      break;
+
+    case DEPENDENCY_JARS:
+      if (value == null) {
+        unset_dependency_jars();
+      } else {
+        set_dependency_jars((List<String>)value);
+      }
+      break;
+
+    case DEPENDENCY_ARTIFACTS:
+      if (value == null) {
+        unset_dependency_artifacts();
+      } else {
+        set_dependency_artifacts((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SPOUTS:
+      return get_spouts();
+
+    case BOLTS:
+      return get_bolts();
+
+    case STATE_SPOUTS:
+      return get_state_spouts();
+
+    case WORKER_HOOKS:
+      return get_worker_hooks();
+
+    case DEPENDENCY_JARS:
+      return get_dependency_jars();
+
+    case DEPENDENCY_ARTIFACTS:
+      return get_dependency_artifacts();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SPOUTS:
+      return is_set_spouts();
+    case BOLTS:
+      return is_set_bolts();
+    case STATE_SPOUTS:
+      return is_set_state_spouts();
+    case WORKER_HOOKS:
+      return is_set_worker_hooks();
+    case DEPENDENCY_JARS:
+      return is_set_dependency_jars();
+    case DEPENDENCY_ARTIFACTS:
+      return is_set_dependency_artifacts();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StormTopology)
+      return this.equals((StormTopology)that);
+    return false;
+  }
+
+  public boolean equals(StormTopology that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_spouts = true && this.is_set_spouts();
+    boolean that_present_spouts = true && that.is_set_spouts();
+    if (this_present_spouts || that_present_spouts) {
+      if (!(this_present_spouts && that_present_spouts))
+        return false;
+      if (!this.spouts.equals(that.spouts))
+        return false;
+    }
+
+    boolean this_present_bolts = true && this.is_set_bolts();
+    boolean that_present_bolts = true && that.is_set_bolts();
+    if (this_present_bolts || that_present_bolts) {
+      if (!(this_present_bolts && that_present_bolts))
+        return false;
+      if (!this.bolts.equals(that.bolts))
+        return false;
+    }
+
+    boolean this_present_state_spouts = true && this.is_set_state_spouts();
+    boolean that_present_state_spouts = true && that.is_set_state_spouts();
+    if (this_present_state_spouts || that_present_state_spouts) {
+      if (!(this_present_state_spouts && that_present_state_spouts))
+        return false;
+      if (!this.state_spouts.equals(that.state_spouts))
+        return false;
+    }
+
+    boolean this_present_worker_hooks = true && this.is_set_worker_hooks();
+    boolean that_present_worker_hooks = true && that.is_set_worker_hooks();
+    if (this_present_worker_hooks || that_present_worker_hooks) {
+      if (!(this_present_worker_hooks && that_present_worker_hooks))
+        return false;
+      if (!this.worker_hooks.equals(that.worker_hooks))
+        return false;
+    }
+
+    boolean this_present_dependency_jars = true && this.is_set_dependency_jars();
+    boolean that_present_dependency_jars = true && that.is_set_dependency_jars();
+    if (this_present_dependency_jars || that_present_dependency_jars) {
+      if (!(this_present_dependency_jars && that_present_dependency_jars))
+        return false;
+      if (!this.dependency_jars.equals(that.dependency_jars))
+        return false;
+    }
+
+    boolean this_present_dependency_artifacts = true && this.is_set_dependency_artifacts();
+    boolean that_present_dependency_artifacts = true && that.is_set_dependency_artifacts();
+    if (this_present_dependency_artifacts || that_present_dependency_artifacts) {
+      if (!(this_present_dependency_artifacts && that_present_dependency_artifacts))
+        return false;
+      if (!this.dependency_artifacts.equals(that.dependency_artifacts))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_spouts = true && (is_set_spouts());
+    list.add(present_spouts);
+    if (present_spouts)
+      list.add(spouts);
+
+    boolean present_bolts = true && (is_set_bolts());
+    list.add(present_bolts);
+    if (present_bolts)
+      list.add(bolts);
+
+    boolean present_state_spouts = true && (is_set_state_spouts());
+    list.add(present_state_spouts);
+    if (present_state_spouts)
+      list.add(state_spouts);
+
+    boolean present_worker_hooks = true && (is_set_worker_hooks());
+    list.add(present_worker_hooks);
+    if (present_worker_hooks)
+      list.add(worker_hooks);
+
+    boolean present_dependency_jars = true && (is_set_dependency_jars());
+    list.add(present_dependency_jars);
+    if (present_dependency_jars)
+      list.add(dependency_jars);
+
+    boolean present_dependency_artifacts = true && (is_set_dependency_artifacts());
+    list.add(present_dependency_artifacts);
+    if (present_dependency_artifacts)
+      list.add(dependency_artifacts);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(StormTopology other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_spouts()).compareTo(other.is_set_spouts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_spouts()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.spouts, other.spouts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_bolts()).compareTo(other.is_set_bolts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_bolts()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bolts, other.bolts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_state_spouts()).compareTo(other.is_set_state_spouts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_state_spouts()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state_spouts, other.state_spouts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_worker_hooks()).compareTo(other.is_set_worker_hooks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_worker_hooks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.worker_hooks, other.worker_hooks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_dependency_jars()).compareTo(other.is_set_dependency_jars());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_dependency_jars()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dependency_jars, other.dependency_jars);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_dependency_artifacts()).compareTo(other.is_set_dependency_artifacts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_dependency_artifacts()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dependency_artifacts, other.dependency_artifacts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StormTopology(");
+    boolean first = true;
+
+    sb.append("spouts:");
+    if (this.spouts == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.spouts);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bolts:");
+    if (this.bolts == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.bolts);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("state_spouts:");
+    if (this.state_spouts == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.state_spouts);
+    }
+    first = false;
+    if (is_set_worker_hooks()) {
+      if (!first) sb.append(", ");
+      sb.append("worker_hooks:");
+      if (this.worker_hooks == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.worker_hooks, sb);
+      }
+      first = false;
+    }
+    if (is_set_dependency_jars()) {
+      if (!first) sb.append(", ");
+      sb.append("dependency_jars:");
+      if (this.dependency_jars == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.dependency_jars);
+      }
+      first = false;
+    }
+    if (is_set_dependency_artifacts()) {
+      if (!first) sb.append(", ");
+      sb.append("dependency_artifacts:");
+      if (this.dependency_artifacts == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.dependency_artifacts);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_spouts()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'spouts' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_bolts()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'bolts' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_state_spouts()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'state_spouts' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class StormTopologyStandardSchemeFactory implements SchemeFactory {
+    public StormTopologyStandardScheme getScheme() {
+      return new StormTopologyStandardScheme();
+    }
+  }
+
+  private static class StormTopologyStandardScheme extends StandardScheme<StormTopology> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, StormTopology struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SPOUTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map44 = iprot.readMapBegin();
+                struct.spouts = new HashMap<String,SpoutSpec>(2*_map44.size);
+                String _key45;
+                SpoutSpec _val46;
+                for (int _i47 = 0; _i47 < _map44.size; ++_i47)
+                {
+                  _key45 = iprot.readString();
+                  _val46 = new SpoutSpec();
+                  _val46.read(iprot);
+                  struct.spouts.put(_key45, _val46);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_spouts_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // BOLTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map48 = iprot.readMapBegin();
+                struct.bolts = new HashMap<String,Bolt>(2*_map48.size);
+                String _key49;
+                Bolt _val50;
+                for (int _i51 = 0; _i51 < _map48.size; ++_i51)
+                {
+                  _key49 = iprot.readString();
+                  _val50 = new Bolt();
+                  _val50.read(iprot);
+                  struct.bolts.put(_key49, _val50);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_bolts_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // STATE_SPOUTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map52 = iprot.readMapBegin();
+                struct.state_spouts = new HashMap<String,StateSpoutSpec>(2*_map52.size);
+                String _key53;
+                StateSpoutSpec _val54;
+                for (int _i55 = 0; _i55 < _map52.size; ++_i55)
+                {
+                  _key53 = iprot.readString();
+                  _val54 = new StateSpoutSpec();
+                  _val54.read(iprot);
+                  struct.state_spouts.put(_key53, _val54);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_state_spouts_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // WORKER_HOOKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list56 = iprot.readListBegin();
+                struct.worker_hooks = new ArrayList<ByteBuffer>(_list56.size);
+                ByteBuffer _elem57;
+                for (int _i58 = 0; _i58 < _list56.size; ++_i58)
+                {
+                  _elem57 = iprot.readBinary();
+                  struct.worker_hooks.add(_elem57);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_worker_hooks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // DEPENDENCY_JARS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list59 = iprot.readListBegin();
+                struct.dependency_jars = new ArrayList<String>(_list59.size);
+                String _elem60;
+                for (int _i61 = 0; _i61 < _list59.size; ++_i61)
+                {
+                  _elem60 = iprot.readString();
+                  struct.dependency_jars.add(_elem60);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_dependency_jars_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // DEPENDENCY_ARTIFACTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list62 = iprot.readListBegin();
+                struct.dependency_artifacts = new ArrayList<String>(_list62.size);
+                String _elem63;
+                for (int _i64 = 0; _i64 < _list62.size; ++_i64)
+                {
+                  _elem63 = iprot.readString();
+                  struct.dependency_artifacts.add(_elem63);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_dependency_artifacts_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, StormTopology struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.spouts != null) {
+        oprot.writeFieldBegin(SPOUTS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.spouts.size()));
+          for (Map.Entry<String, SpoutSpec> _iter65 : struct.spouts.entrySet())
+          {
+            oprot.writeString(_iter65.getKey());
+            _iter65.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.bolts != null) {
+        oprot.writeFieldBegin(BOLTS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.bolts.size()));
+          for (Map.Entry<String, Bolt> _iter66 : struct.bolts.entrySet())
+          {
+            oprot.writeString(_iter66.getKey());
+            _iter66.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.state_spouts != null) {
+        oprot.writeFieldBegin(STATE_SPOUTS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.state_spouts.size()));
+          for (Map.Entry<String, StateSpoutSpec> _iter67 : struct.state_spouts.entrySet())
+          {
+            oprot.writeString(_iter67.getKey());
+            _iter67.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.worker_hooks != null) {
+        if (struct.is_set_worker_hooks()) {
+          oprot.writeFieldBegin(WORKER_HOOKS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.worker_hooks.size()));
+            for (ByteBuffer _iter68 : struct.worker_hooks)
+            {
+              oprot.writeBinary(_iter68);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.dependency_jars != null) {
+        if (struct.is_set_dependency_jars()) {
+          oprot.writeFieldBegin(DEPENDENCY_JARS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.dependency_jars.size()));
+            for (String _iter69 : struct.dependency_jars)
+            {
+              oprot.writeString(_iter69);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.dependency_artifacts != null) {
+        if (struct.is_set_dependency_artifacts()) {
+          oprot.writeFieldBegin(DEPENDENCY_ARTIFACTS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.dependency_artifacts.size()));
+            for (String _iter70 : struct.dependency_artifacts)
+            {
+              oprot.writeString(_iter70);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class StormTopologyTupleSchemeFactory implements SchemeFactory {
+    public StormTopologyTupleScheme getScheme() {
+      return new StormTopologyTupleScheme();
+    }
+  }
+
+  private static class StormTopologyTupleScheme extends TupleScheme<StormTopology> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, StormTopology struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.spouts.size());
+        for (Map.Entry<String, SpoutSpec> _iter71 : struct.spouts.entrySet())
+        {
+          oprot.writeString(_iter71.getKey());
+          _iter71.getValue().write(oprot);
+        }
+      }
+      {
+        oprot.writeI32(struct.bolts.size());
+        for (Map.Entry<String, Bolt> _iter72 : struct.bolts.entrySet())
+        {
+          oprot.writeString(_iter72.getKey());
+          _iter72.getValue().write(oprot);
+        }
+      }
+      {
+        oprot.writeI32(struct.state_spouts.size());
+        for (Map.Entry<String, StateSpoutSpec> _iter73 : struct.state_spouts.entrySet())
+        {
+          oprot.writeString(_iter73.getKey());
+          _iter73.getValue().write(oprot);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_worker_hooks()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_dependency_jars()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_dependency_artifacts()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_worker_hooks()) {
+        {
+          oprot.writeI32(struct.worker_hooks.size());
+          for (ByteBuffer _iter74 : struct.worker_hooks)
+          {
+            oprot.writeBinary(_iter74);
+          }
+        }
+      }
+      if (struct.is_set_dependency_jars()) {
+        {
+          oprot.writeI32(struct.dependency_jars.size());
+          for (String _iter75 : struct.dependency_jars)
+          {
+            oprot.writeString(_iter75);
+          }
+        }
+      }
+      if (struct.is_set_dependency_artifacts()) {
+        {
+          oprot.writeI32(struct.dependency_artifacts.size());
+          for (String _iter76 : struct.dependency_artifacts)
+          {
+            oprot.writeString(_iter76);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, StormTopology struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map77 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.spouts = new HashMap<String,SpoutSpec>(2*_map77.size);
+        String _key78;
+        SpoutSpec _val79;
+        for (int _i80 = 0; _i80 < _map77.size; ++_i80)
+        {
+          _key78 = iprot.readString();
+          _val79 = new SpoutSpec();
+          _val79.read(iprot);
+          struct.spouts.put(_key78, _val79);
+        }
+      }
+      struct.set_spouts_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map81 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.bolts = new HashMap<String,Bolt>(2*_map81.size);
+        String _key82;
+        Bolt _val83;
+        for (int _i84 = 0; _i84 < _map81.size; ++_i84)
+        {
+          _key82 = iprot.readString();
+          _val83 = new Bolt();
+          _val83.read(iprot);
+          struct.bolts.put(_key82, _val83);
+        }
+      }
+      struct.set_bolts_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map85 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.state_spouts = new HashMap<String,StateSpoutSpec>(2*_map85.size);
+        String _key86;
+        StateSpoutSpec _val87;
+        for (int _i88 = 0; _i88 < _map85.size; ++_i88)
+        {
+          _key86 = iprot.readString();
+          _val87 = new StateSpoutSpec();
+          _val87.read(iprot);
+          struct.state_spouts.put(_key86, _val87);
+        }
+      }
+      struct.set_state_spouts_isSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list89 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.worker_hooks = new ArrayList<ByteBuffer>(_list89.size);
+          ByteBuffer _elem90;
+          for (int _i91 = 0; _i91 < _list89.size; ++_i91)
+          {
+            _elem90 = iprot.readBinary();
+            struct.worker_hooks.add(_elem90);
+          }
+        }
+        struct.set_worker_hooks_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list92 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.dependency_jars = new ArrayList<String>(_list92.size);
+          String _elem93;
+          for (int _i94 = 0; _i94 < _list92.size; ++_i94)
+          {
+            _elem93 = iprot.readString();
+            struct.dependency_jars.add(_elem93);
+          }
+        }
+        struct.set_dependency_jars_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list95 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.dependency_artifacts = new ArrayList<String>(_list95.size);
+          String _elem96;
+          for (int _i97 = 0; _i97 < _list95.size; ++_i97)
+          {
+            _elem96 = iprot.readString();
+            struct.dependency_artifacts.add(_elem96);
+          }
+        }
+        struct.set_dependency_artifacts_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/StreamInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/StreamInfo.java b/storm-client/src/jvm/org/apache/storm/generated/StreamInfo.java
new file mode 100644
index 0000000..bb78631
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/StreamInfo.java
@@ -0,0 +1,554 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class StreamInfo implements org.apache.thrift.TBase<StreamInfo, StreamInfo._Fields>, java.io.Serializable, Cloneable, Comparable<StreamInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StreamInfo");
+
+  private static final org.apache.thrift.protocol.TField OUTPUT_FIELDS_FIELD_DESC = new org.apache.thrift.protocol.TField("output_fields", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift.protocol.TField("direct", org.apache.thrift.protocol.TType.BOOL, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new StreamInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new StreamInfoTupleSchemeFactory());
+  }
+
+  private List<String> output_fields; // required
+  private boolean direct; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    OUTPUT_FIELDS((short)1, "output_fields"),
+    DIRECT((short)2, "direct");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // OUTPUT_FIELDS
+          return OUTPUT_FIELDS;
+        case 2: // DIRECT
+          return DIRECT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __DIRECT_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.OUTPUT_FIELDS, new org.apache.thrift.meta_data.FieldMetaData("output_fields", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.DIRECT, new org.apache.thrift.meta_data.FieldMetaData("direct", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StreamInfo.class, metaDataMap);
+  }
+
+  public StreamInfo() {
+  }
+
+  public StreamInfo(
+    List<String> output_fields,
+    boolean direct)
+  {
+    this();
+    this.output_fields = output_fields;
+    this.direct = direct;
+    set_direct_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StreamInfo(StreamInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_output_fields()) {
+      List<String> __this__output_fields = new ArrayList<String>(other.output_fields);
+      this.output_fields = __this__output_fields;
+    }
+    this.direct = other.direct;
+  }
+
+  public StreamInfo deepCopy() {
+    return new StreamInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.output_fields = null;
+    set_direct_isSet(false);
+    this.direct = false;
+  }
+
+  public int get_output_fields_size() {
+    return (this.output_fields == null) ? 0 : this.output_fields.size();
+  }
+
+  public java.util.Iterator<String> get_output_fields_iterator() {
+    return (this.output_fields == null) ? null : this.output_fields.iterator();
+  }
+
+  public void add_to_output_fields(String elem) {
+    if (this.output_fields == null) {
+      this.output_fields = new ArrayList<String>();
+    }
+    this.output_fields.add(elem);
+  }
+
+  public List<String> get_output_fields() {
+    return this.output_fields;
+  }
+
+  public void set_output_fields(List<String> output_fields) {
+    this.output_fields = output_fields;
+  }
+
+  public void unset_output_fields() {
+    this.output_fields = null;
+  }
+
+  /** Returns true if field output_fields is set (has been assigned a value) and false otherwise */
+  public boolean is_set_output_fields() {
+    return this.output_fields != null;
+  }
+
+  public void set_output_fields_isSet(boolean value) {
+    if (!value) {
+      this.output_fields = null;
+    }
+  }
+
+  public boolean is_direct() {
+    return this.direct;
+  }
+
+  public void set_direct(boolean direct) {
+    this.direct = direct;
+    set_direct_isSet(true);
+  }
+
+  public void unset_direct() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DIRECT_ISSET_ID);
+  }
+
+  /** Returns true if field direct is set (has been assigned a value) and false otherwise */
+  public boolean is_set_direct() {
+    return EncodingUtils.testBit(__isset_bitfield, __DIRECT_ISSET_ID);
+  }
+
+  public void set_direct_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DIRECT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case OUTPUT_FIELDS:
+      if (value == null) {
+        unset_output_fields();
+      } else {
+        set_output_fields((List<String>)value);
+      }
+      break;
+
+    case DIRECT:
+      if (value == null) {
+        unset_direct();
+      } else {
+        set_direct((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case OUTPUT_FIELDS:
+      return get_output_fields();
+
+    case DIRECT:
+      return is_direct();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case OUTPUT_FIELDS:
+      return is_set_output_fields();
+    case DIRECT:
+      return is_set_direct();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StreamInfo)
+      return this.equals((StreamInfo)that);
+    return false;
+  }
+
+  public boolean equals(StreamInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_output_fields = true && this.is_set_output_fields();
+    boolean that_present_output_fields = true && that.is_set_output_fields();
+    if (this_present_output_fields || that_present_output_fields) {
+      if (!(this_present_output_fields && that_present_output_fields))
+        return false;
+      if (!this.output_fields.equals(that.output_fields))
+        return false;
+    }
+
+    boolean this_present_direct = true;
+    boolean that_present_direct = true;
+    if (this_present_direct || that_present_direct) {
+      if (!(this_present_direct && that_present_direct))
+        return false;
+      if (this.direct != that.direct)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_output_fields = true && (is_set_output_fields());
+    list.add(present_output_fields);
+    if (present_output_fields)
+      list.add(output_fields);
+
+    boolean present_direct = true;
+    list.add(present_direct);
+    if (present_direct)
+      list.add(direct);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(StreamInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_output_fields()).compareTo(other.is_set_output_fields());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_output_fields()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.output_fields, other.output_fields);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_direct()).compareTo(other.is_set_direct());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_direct()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.direct, other.direct);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StreamInfo(");
+    boolean first = true;
+
+    sb.append("output_fields:");
+    if (this.output_fields == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.output_fields);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("direct:");
+    sb.append(this.direct);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_output_fields()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'output_fields' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_direct()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'direct' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class StreamInfoStandardSchemeFactory implements SchemeFactory {
+    public StreamInfoStandardScheme getScheme() {
+      return new StreamInfoStandardScheme();
+    }
+  }
+
+  private static class StreamInfoStandardScheme extends StandardScheme<StreamInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, StreamInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // OUTPUT_FIELDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list16 = iprot.readListBegin();
+                struct.output_fields = new ArrayList<String>(_list16.size);
+                String _elem17;
+                for (int _i18 = 0; _i18 < _list16.size; ++_i18)
+                {
+                  _elem17 = iprot.readString();
+                  struct.output_fields.add(_elem17);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_output_fields_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DIRECT
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.direct = iprot.readBool();
+              struct.set_direct_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, StreamInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.output_fields != null) {
+        oprot.writeFieldBegin(OUTPUT_FIELDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.output_fields.size()));
+          for (String _iter19 : struct.output_fields)
+          {
+            oprot.writeString(_iter19);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(DIRECT_FIELD_DESC);
+      oprot.writeBool(struct.direct);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class StreamInfoTupleSchemeFactory implements SchemeFactory {
+    public StreamInfoTupleScheme getScheme() {
+      return new StreamInfoTupleScheme();
+    }
+  }
+
+  private static class StreamInfoTupleScheme extends TupleScheme<StreamInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, StreamInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.output_fields.size());
+        for (String _iter20 : struct.output_fields)
+        {
+          oprot.writeString(_iter20);
+        }
+      }
+      oprot.writeBool(struct.direct);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, StreamInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list21 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.output_fields = new ArrayList<String>(_list21.size);
+        String _elem22;
+        for (int _i23 = 0; _i23 < _list21.size; ++_i23)
+        {
+          _elem22 = iprot.readString();
+          struct.output_fields.add(_elem22);
+        }
+      }
+      struct.set_output_fields_isSet(true);
+      struct.direct = iprot.readBool();
+      struct.set_direct_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/SubmitOptions.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/SubmitOptions.java b/storm-client/src/jvm/org/apache/storm/generated/SubmitOptions.java
new file mode 100644
index 0000000..2287244
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/SubmitOptions.java
@@ -0,0 +1,533 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, SubmitOptions._Fields>, java.io.Serializable, Cloneable, Comparable<SubmitOptions> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SubmitOptions");
+
+  private static final org.apache.thrift.protocol.TField INITIAL_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("initial_status", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SubmitOptionsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SubmitOptionsTupleSchemeFactory());
+  }
+
+  private TopologyInitialStatus initial_status; // required
+  private Credentials creds; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see TopologyInitialStatus
+     */
+    INITIAL_STATUS((short)1, "initial_status"),
+    CREDS((short)2, "creds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // INITIAL_STATUS
+          return INITIAL_STATUS;
+        case 2: // CREDS
+          return CREDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CREDS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.INITIAL_STATUS, new org.apache.thrift.meta_data.FieldMetaData("initial_status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyInitialStatus.class)));
+    tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Credentials.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SubmitOptions.class, metaDataMap);
+  }
+
+  public SubmitOptions() {
+  }
+
+  public SubmitOptions(
+    TopologyInitialStatus initial_status)
+  {
+    this();
+    this.initial_status = initial_status;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SubmitOptions(SubmitOptions other) {
+    if (other.is_set_initial_status()) {
+      this.initial_status = other.initial_status;
+    }
+    if (other.is_set_creds()) {
+      this.creds = new Credentials(other.creds);
+    }
+  }
+
+  public SubmitOptions deepCopy() {
+    return new SubmitOptions(this);
+  }
+
+  @Override
+  public void clear() {
+    this.initial_status = null;
+    this.creds = null;
+  }
+
+  /**
+   * 
+   * @see TopologyInitialStatus
+   */
+  public TopologyInitialStatus get_initial_status() {
+    return this.initial_status;
+  }
+
+  /**
+   * 
+   * @see TopologyInitialStatus
+   */
+  public void set_initial_status(TopologyInitialStatus initial_status) {
+    this.initial_status = initial_status;
+  }
+
+  public void unset_initial_status() {
+    this.initial_status = null;
+  }
+
+  /** Returns true if field initial_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_initial_status() {
+    return this.initial_status != null;
+  }
+
+  public void set_initial_status_isSet(boolean value) {
+    if (!value) {
+      this.initial_status = null;
+    }
+  }
+
+  public Credentials get_creds() {
+    return this.creds;
+  }
+
+  public void set_creds(Credentials creds) {
+    this.creds = creds;
+  }
+
+  public void unset_creds() {
+    this.creds = null;
+  }
+
+  /** Returns true if field creds is set (has been assigned a value) and false otherwise */
+  public boolean is_set_creds() {
+    return this.creds != null;
+  }
+
+  public void set_creds_isSet(boolean value) {
+    if (!value) {
+      this.creds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case INITIAL_STATUS:
+      if (value == null) {
+        unset_initial_status();
+      } else {
+        set_initial_status((TopologyInitialStatus)value);
+      }
+      break;
+
+    case CREDS:
+      if (value == null) {
+        unset_creds();
+      } else {
+        set_creds((Credentials)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case INITIAL_STATUS:
+      return get_initial_status();
+
+    case CREDS:
+      return get_creds();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case INITIAL_STATUS:
+      return is_set_initial_status();
+    case CREDS:
+      return is_set_creds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SubmitOptions)
+      return this.equals((SubmitOptions)that);
+    return false;
+  }
+
+  public boolean equals(SubmitOptions that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_initial_status = true && this.is_set_initial_status();
+    boolean that_present_initial_status = true && that.is_set_initial_status();
+    if (this_present_initial_status || that_present_initial_status) {
+      if (!(this_present_initial_status && that_present_initial_status))
+        return false;
+      if (!this.initial_status.equals(that.initial_status))
+        return false;
+    }
+
+    boolean this_present_creds = true && this.is_set_creds();
+    boolean that_present_creds = true && that.is_set_creds();
+    if (this_present_creds || that_present_creds) {
+      if (!(this_present_creds && that_present_creds))
+        return false;
+      if (!this.creds.equals(that.creds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_initial_status = true && (is_set_initial_status());
+    list.add(present_initial_status);
+    if (present_initial_status)
+      list.add(initial_status.getValue());
+
+    boolean present_creds = true && (is_set_creds());
+    list.add(present_creds);
+    if (present_creds)
+      list.add(creds);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SubmitOptions other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_initial_status()).compareTo(other.is_set_initial_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_initial_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.initial_status, other.initial_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_creds()).compareTo(other.is_set_creds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_creds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, other.creds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SubmitOptions(");
+    boolean first = true;
+
+    sb.append("initial_status:");
+    if (this.initial_status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.initial_status);
+    }
+    first = false;
+    if (is_set_creds()) {
+      if (!first) sb.append(", ");
+      sb.append("creds:");
+      if (this.creds == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.creds);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_initial_status()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'initial_status' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (creds != null) {
+      creds.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SubmitOptionsStandardSchemeFactory implements SchemeFactory {
+    public SubmitOptionsStandardScheme getScheme() {
+      return new SubmitOptionsStandardScheme();
+    }
+  }
+
+  private static class SubmitOptionsStandardScheme extends StandardScheme<SubmitOptions> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SubmitOptions struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // INITIAL_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.initial_status = org.apache.storm.generated.TopologyInitialStatus.findByValue(iprot.readI32());
+              struct.set_initial_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // CREDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.creds = new Credentials();
+              struct.creds.read(iprot);
+              struct.set_creds_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SubmitOptions struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.initial_status != null) {
+        oprot.writeFieldBegin(INITIAL_STATUS_FIELD_DESC);
+        oprot.writeI32(struct.initial_status.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.creds != null) {
+        if (struct.is_set_creds()) {
+          oprot.writeFieldBegin(CREDS_FIELD_DESC);
+          struct.creds.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SubmitOptionsTupleSchemeFactory implements SchemeFactory {
+    public SubmitOptionsTupleScheme getScheme() {
+      return new SubmitOptionsTupleScheme();
+    }
+  }
+
+  private static class SubmitOptionsTupleScheme extends TupleScheme<SubmitOptions> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SubmitOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.initial_status.getValue());
+      BitSet optionals = new BitSet();
+      if (struct.is_set_creds()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_creds()) {
+        struct.creds.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SubmitOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.initial_status = org.apache.storm.generated.TopologyInitialStatus.findByValue(iprot.readI32());
+      struct.set_initial_status_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.creds = new Credentials();
+        struct.creds.read(iprot);
+        struct.set_creds_isSet(true);
+      }
+    }
+  }
+
+}
+


[33/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ComponentAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ComponentAggregateStats.java b/storm-client/src/jvm/org/apache/storm/generated/ComponentAggregateStats.java
new file mode 100644
index 0000000..2c68331
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ComponentAggregateStats.java
@@ -0,0 +1,752 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ComponentAggregateStats implements org.apache.thrift.TBase<ComponentAggregateStats, ComponentAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<ComponentAggregateStats> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentAggregateStats");
+
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField COMMON_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("common_stats", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField SPECIFIC_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("specific_stats", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField LAST_ERROR_FIELD_DESC = new org.apache.thrift.protocol.TField("last_error", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ComponentAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ComponentAggregateStatsTupleSchemeFactory());
+  }
+
+  private ComponentType type; // optional
+  private CommonAggregateStats common_stats; // optional
+  private SpecificAggregateStats specific_stats; // optional
+  private ErrorInfo last_error; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see ComponentType
+     */
+    TYPE((short)1, "type"),
+    COMMON_STATS((short)2, "common_stats"),
+    SPECIFIC_STATS((short)3, "specific_stats"),
+    LAST_ERROR((short)4, "last_error");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TYPE
+          return TYPE;
+        case 2: // COMMON_STATS
+          return COMMON_STATS;
+        case 3: // SPECIFIC_STATS
+          return SPECIFIC_STATS;
+        case 4: // LAST_ERROR
+          return LAST_ERROR;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.TYPE,_Fields.COMMON_STATS,_Fields.SPECIFIC_STATS,_Fields.LAST_ERROR};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ComponentType.class)));
+    tmpMap.put(_Fields.COMMON_STATS, new org.apache.thrift.meta_data.FieldMetaData("common_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CommonAggregateStats.class)));
+    tmpMap.put(_Fields.SPECIFIC_STATS, new org.apache.thrift.meta_data.FieldMetaData("specific_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpecificAggregateStats.class)));
+    tmpMap.put(_Fields.LAST_ERROR, new org.apache.thrift.meta_data.FieldMetaData("last_error", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentAggregateStats.class, metaDataMap);
+  }
+
+  public ComponentAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ComponentAggregateStats(ComponentAggregateStats other) {
+    if (other.is_set_type()) {
+      this.type = other.type;
+    }
+    if (other.is_set_common_stats()) {
+      this.common_stats = new CommonAggregateStats(other.common_stats);
+    }
+    if (other.is_set_specific_stats()) {
+      this.specific_stats = new SpecificAggregateStats(other.specific_stats);
+    }
+    if (other.is_set_last_error()) {
+      this.last_error = new ErrorInfo(other.last_error);
+    }
+  }
+
+  public ComponentAggregateStats deepCopy() {
+    return new ComponentAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.type = null;
+    this.common_stats = null;
+    this.specific_stats = null;
+    this.last_error = null;
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public ComponentType get_type() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public void set_type(ComponentType type) {
+    this.type = type;
+  }
+
+  public void unset_type() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_type() {
+    return this.type != null;
+  }
+
+  public void set_type_isSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public CommonAggregateStats get_common_stats() {
+    return this.common_stats;
+  }
+
+  public void set_common_stats(CommonAggregateStats common_stats) {
+    this.common_stats = common_stats;
+  }
+
+  public void unset_common_stats() {
+    this.common_stats = null;
+  }
+
+  /** Returns true if field common_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_common_stats() {
+    return this.common_stats != null;
+  }
+
+  public void set_common_stats_isSet(boolean value) {
+    if (!value) {
+      this.common_stats = null;
+    }
+  }
+
+  public SpecificAggregateStats get_specific_stats() {
+    return this.specific_stats;
+  }
+
+  public void set_specific_stats(SpecificAggregateStats specific_stats) {
+    this.specific_stats = specific_stats;
+  }
+
+  public void unset_specific_stats() {
+    this.specific_stats = null;
+  }
+
+  /** Returns true if field specific_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_specific_stats() {
+    return this.specific_stats != null;
+  }
+
+  public void set_specific_stats_isSet(boolean value) {
+    if (!value) {
+      this.specific_stats = null;
+    }
+  }
+
+  public ErrorInfo get_last_error() {
+    return this.last_error;
+  }
+
+  public void set_last_error(ErrorInfo last_error) {
+    this.last_error = last_error;
+  }
+
+  public void unset_last_error() {
+    this.last_error = null;
+  }
+
+  /** Returns true if field last_error is set (has been assigned a value) and false otherwise */
+  public boolean is_set_last_error() {
+    return this.last_error != null;
+  }
+
+  public void set_last_error_isSet(boolean value) {
+    if (!value) {
+      this.last_error = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TYPE:
+      if (value == null) {
+        unset_type();
+      } else {
+        set_type((ComponentType)value);
+      }
+      break;
+
+    case COMMON_STATS:
+      if (value == null) {
+        unset_common_stats();
+      } else {
+        set_common_stats((CommonAggregateStats)value);
+      }
+      break;
+
+    case SPECIFIC_STATS:
+      if (value == null) {
+        unset_specific_stats();
+      } else {
+        set_specific_stats((SpecificAggregateStats)value);
+      }
+      break;
+
+    case LAST_ERROR:
+      if (value == null) {
+        unset_last_error();
+      } else {
+        set_last_error((ErrorInfo)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TYPE:
+      return get_type();
+
+    case COMMON_STATS:
+      return get_common_stats();
+
+    case SPECIFIC_STATS:
+      return get_specific_stats();
+
+    case LAST_ERROR:
+      return get_last_error();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TYPE:
+      return is_set_type();
+    case COMMON_STATS:
+      return is_set_common_stats();
+    case SPECIFIC_STATS:
+      return is_set_specific_stats();
+    case LAST_ERROR:
+      return is_set_last_error();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ComponentAggregateStats)
+      return this.equals((ComponentAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(ComponentAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_type = true && this.is_set_type();
+    boolean that_present_type = true && that.is_set_type();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_common_stats = true && this.is_set_common_stats();
+    boolean that_present_common_stats = true && that.is_set_common_stats();
+    if (this_present_common_stats || that_present_common_stats) {
+      if (!(this_present_common_stats && that_present_common_stats))
+        return false;
+      if (!this.common_stats.equals(that.common_stats))
+        return false;
+    }
+
+    boolean this_present_specific_stats = true && this.is_set_specific_stats();
+    boolean that_present_specific_stats = true && that.is_set_specific_stats();
+    if (this_present_specific_stats || that_present_specific_stats) {
+      if (!(this_present_specific_stats && that_present_specific_stats))
+        return false;
+      if (!this.specific_stats.equals(that.specific_stats))
+        return false;
+    }
+
+    boolean this_present_last_error = true && this.is_set_last_error();
+    boolean that_present_last_error = true && that.is_set_last_error();
+    if (this_present_last_error || that_present_last_error) {
+      if (!(this_present_last_error && that_present_last_error))
+        return false;
+      if (!this.last_error.equals(that.last_error))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_type = true && (is_set_type());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
+    boolean present_common_stats = true && (is_set_common_stats());
+    list.add(present_common_stats);
+    if (present_common_stats)
+      list.add(common_stats);
+
+    boolean present_specific_stats = true && (is_set_specific_stats());
+    list.add(present_specific_stats);
+    if (present_specific_stats)
+      list.add(specific_stats);
+
+    boolean present_last_error = true && (is_set_last_error());
+    list.add(present_last_error);
+    if (present_last_error)
+      list.add(last_error);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ComponentAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common_stats()).compareTo(other.is_set_common_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common_stats, other.common_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_specific_stats()).compareTo(other.is_set_specific_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_specific_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.specific_stats, other.specific_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_last_error()).compareTo(other.is_set_last_error());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_last_error()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.last_error, other.last_error);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ComponentAggregateStats(");
+    boolean first = true;
+
+    if (is_set_type()) {
+      sb.append("type:");
+      if (this.type == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.type);
+      }
+      first = false;
+    }
+    if (is_set_common_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("common_stats:");
+      if (this.common_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.common_stats);
+      }
+      first = false;
+    }
+    if (is_set_specific_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("specific_stats:");
+      if (this.specific_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.specific_stats);
+      }
+      first = false;
+    }
+    if (is_set_last_error()) {
+      if (!first) sb.append(", ");
+      sb.append("last_error:");
+      if (this.last_error == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.last_error);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (common_stats != null) {
+      common_stats.validate();
+    }
+    if (last_error != null) {
+      last_error.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ComponentAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public ComponentAggregateStatsStandardScheme getScheme() {
+      return new ComponentAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class ComponentAggregateStatsStandardScheme extends StandardScheme<ComponentAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = org.apache.storm.generated.ComponentType.findByValue(iprot.readI32());
+              struct.set_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMMON_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.common_stats = new CommonAggregateStats();
+              struct.common_stats.read(iprot);
+              struct.set_common_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // SPECIFIC_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.specific_stats = new SpecificAggregateStats();
+              struct.specific_stats.read(iprot);
+              struct.set_specific_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // LAST_ERROR
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.last_error = new ErrorInfo();
+              struct.last_error.read(iprot);
+              struct.set_last_error_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.type != null) {
+        if (struct.is_set_type()) {
+          oprot.writeFieldBegin(TYPE_FIELD_DESC);
+          oprot.writeI32(struct.type.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.common_stats != null) {
+        if (struct.is_set_common_stats()) {
+          oprot.writeFieldBegin(COMMON_STATS_FIELD_DESC);
+          struct.common_stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.specific_stats != null) {
+        if (struct.is_set_specific_stats()) {
+          oprot.writeFieldBegin(SPECIFIC_STATS_FIELD_DESC);
+          struct.specific_stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.last_error != null) {
+        if (struct.is_set_last_error()) {
+          oprot.writeFieldBegin(LAST_ERROR_FIELD_DESC);
+          struct.last_error.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ComponentAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public ComponentAggregateStatsTupleScheme getScheme() {
+      return new ComponentAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class ComponentAggregateStatsTupleScheme extends TupleScheme<ComponentAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_type()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_common_stats()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_specific_stats()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_last_error()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_type()) {
+        oprot.writeI32(struct.type.getValue());
+      }
+      if (struct.is_set_common_stats()) {
+        struct.common_stats.write(oprot);
+      }
+      if (struct.is_set_specific_stats()) {
+        struct.specific_stats.write(oprot);
+      }
+      if (struct.is_set_last_error()) {
+        struct.last_error.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.type = org.apache.storm.generated.ComponentType.findByValue(iprot.readI32());
+        struct.set_type_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.common_stats = new CommonAggregateStats();
+        struct.common_stats.read(iprot);
+        struct.set_common_stats_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.specific_stats = new SpecificAggregateStats();
+        struct.specific_stats.read(iprot);
+        struct.set_specific_stats_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.last_error = new ErrorInfo();
+        struct.last_error.read(iprot);
+        struct.set_last_error_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ComponentCommon.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ComponentCommon.java b/storm-client/src/jvm/org/apache/storm/generated/ComponentCommon.java
new file mode 100644
index 0000000..420ad60
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ComponentCommon.java
@@ -0,0 +1,852 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ComponentCommon implements org.apache.thrift.TBase<ComponentCommon, ComponentCommon._Fields>, java.io.Serializable, Cloneable, Comparable<ComponentCommon> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentCommon");
+
+  private static final org.apache.thrift.protocol.TField INPUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("inputs", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField STREAMS_FIELD_DESC = new org.apache.thrift.protocol.TField("streams", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField PARALLELISM_HINT_FIELD_DESC = new org.apache.thrift.protocol.TField("parallelism_hint", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("json_conf", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ComponentCommonStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ComponentCommonTupleSchemeFactory());
+  }
+
+  private Map<GlobalStreamId,Grouping> inputs; // required
+  private Map<String,StreamInfo> streams; // required
+  private int parallelism_hint; // optional
+  private String json_conf; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    INPUTS((short)1, "inputs"),
+    STREAMS((short)2, "streams"),
+    PARALLELISM_HINT((short)3, "parallelism_hint"),
+    JSON_CONF((short)4, "json_conf");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // INPUTS
+          return INPUTS;
+        case 2: // STREAMS
+          return STREAMS;
+        case 3: // PARALLELISM_HINT
+          return PARALLELISM_HINT;
+        case 4: // JSON_CONF
+          return JSON_CONF;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PARALLELISM_HINT_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.PARALLELISM_HINT,_Fields.JSON_CONF};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.INPUTS, new org.apache.thrift.meta_data.FieldMetaData("inputs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Grouping.class))));
+    tmpMap.put(_Fields.STREAMS, new org.apache.thrift.meta_data.FieldMetaData("streams", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StreamInfo.class))));
+    tmpMap.put(_Fields.PARALLELISM_HINT, new org.apache.thrift.meta_data.FieldMetaData("parallelism_hint", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift.meta_data.FieldMetaData("json_conf", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentCommon.class, metaDataMap);
+  }
+
+  public ComponentCommon() {
+  }
+
+  public ComponentCommon(
+    Map<GlobalStreamId,Grouping> inputs,
+    Map<String,StreamInfo> streams)
+  {
+    this();
+    this.inputs = inputs;
+    this.streams = streams;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ComponentCommon(ComponentCommon other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_inputs()) {
+      Map<GlobalStreamId,Grouping> __this__inputs = new HashMap<GlobalStreamId,Grouping>(other.inputs.size());
+      for (Map.Entry<GlobalStreamId, Grouping> other_element : other.inputs.entrySet()) {
+
+        GlobalStreamId other_element_key = other_element.getKey();
+        Grouping other_element_value = other_element.getValue();
+
+        GlobalStreamId __this__inputs_copy_key = new GlobalStreamId(other_element_key);
+
+        Grouping __this__inputs_copy_value = new Grouping(other_element_value);
+
+        __this__inputs.put(__this__inputs_copy_key, __this__inputs_copy_value);
+      }
+      this.inputs = __this__inputs;
+    }
+    if (other.is_set_streams()) {
+      Map<String,StreamInfo> __this__streams = new HashMap<String,StreamInfo>(other.streams.size());
+      for (Map.Entry<String, StreamInfo> other_element : other.streams.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        StreamInfo other_element_value = other_element.getValue();
+
+        String __this__streams_copy_key = other_element_key;
+
+        StreamInfo __this__streams_copy_value = new StreamInfo(other_element_value);
+
+        __this__streams.put(__this__streams_copy_key, __this__streams_copy_value);
+      }
+      this.streams = __this__streams;
+    }
+    this.parallelism_hint = other.parallelism_hint;
+    if (other.is_set_json_conf()) {
+      this.json_conf = other.json_conf;
+    }
+  }
+
+  public ComponentCommon deepCopy() {
+    return new ComponentCommon(this);
+  }
+
+  @Override
+  public void clear() {
+    this.inputs = null;
+    this.streams = null;
+    set_parallelism_hint_isSet(false);
+    this.parallelism_hint = 0;
+    this.json_conf = null;
+  }
+
+  public int get_inputs_size() {
+    return (this.inputs == null) ? 0 : this.inputs.size();
+  }
+
+  public void put_to_inputs(GlobalStreamId key, Grouping val) {
+    if (this.inputs == null) {
+      this.inputs = new HashMap<GlobalStreamId,Grouping>();
+    }
+    this.inputs.put(key, val);
+  }
+
+  public Map<GlobalStreamId,Grouping> get_inputs() {
+    return this.inputs;
+  }
+
+  public void set_inputs(Map<GlobalStreamId,Grouping> inputs) {
+    this.inputs = inputs;
+  }
+
+  public void unset_inputs() {
+    this.inputs = null;
+  }
+
+  /** Returns true if field inputs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_inputs() {
+    return this.inputs != null;
+  }
+
+  public void set_inputs_isSet(boolean value) {
+    if (!value) {
+      this.inputs = null;
+    }
+  }
+
+  public int get_streams_size() {
+    return (this.streams == null) ? 0 : this.streams.size();
+  }
+
+  public void put_to_streams(String key, StreamInfo val) {
+    if (this.streams == null) {
+      this.streams = new HashMap<String,StreamInfo>();
+    }
+    this.streams.put(key, val);
+  }
+
+  public Map<String,StreamInfo> get_streams() {
+    return this.streams;
+  }
+
+  public void set_streams(Map<String,StreamInfo> streams) {
+    this.streams = streams;
+  }
+
+  public void unset_streams() {
+    this.streams = null;
+  }
+
+  /** Returns true if field streams is set (has been assigned a value) and false otherwise */
+  public boolean is_set_streams() {
+    return this.streams != null;
+  }
+
+  public void set_streams_isSet(boolean value) {
+    if (!value) {
+      this.streams = null;
+    }
+  }
+
+  public int get_parallelism_hint() {
+    return this.parallelism_hint;
+  }
+
+  public void set_parallelism_hint(int parallelism_hint) {
+    this.parallelism_hint = parallelism_hint;
+    set_parallelism_hint_isSet(true);
+  }
+
+  public void unset_parallelism_hint() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID);
+  }
+
+  /** Returns true if field parallelism_hint is set (has been assigned a value) and false otherwise */
+  public boolean is_set_parallelism_hint() {
+    return EncodingUtils.testBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID);
+  }
+
+  public void set_parallelism_hint_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID, value);
+  }
+
+  public String get_json_conf() {
+    return this.json_conf;
+  }
+
+  public void set_json_conf(String json_conf) {
+    this.json_conf = json_conf;
+  }
+
+  public void unset_json_conf() {
+    this.json_conf = null;
+  }
+
+  /** Returns true if field json_conf is set (has been assigned a value) and false otherwise */
+  public boolean is_set_json_conf() {
+    return this.json_conf != null;
+  }
+
+  public void set_json_conf_isSet(boolean value) {
+    if (!value) {
+      this.json_conf = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case INPUTS:
+      if (value == null) {
+        unset_inputs();
+      } else {
+        set_inputs((Map<GlobalStreamId,Grouping>)value);
+      }
+      break;
+
+    case STREAMS:
+      if (value == null) {
+        unset_streams();
+      } else {
+        set_streams((Map<String,StreamInfo>)value);
+      }
+      break;
+
+    case PARALLELISM_HINT:
+      if (value == null) {
+        unset_parallelism_hint();
+      } else {
+        set_parallelism_hint((Integer)value);
+      }
+      break;
+
+    case JSON_CONF:
+      if (value == null) {
+        unset_json_conf();
+      } else {
+        set_json_conf((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case INPUTS:
+      return get_inputs();
+
+    case STREAMS:
+      return get_streams();
+
+    case PARALLELISM_HINT:
+      return get_parallelism_hint();
+
+    case JSON_CONF:
+      return get_json_conf();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case INPUTS:
+      return is_set_inputs();
+    case STREAMS:
+      return is_set_streams();
+    case PARALLELISM_HINT:
+      return is_set_parallelism_hint();
+    case JSON_CONF:
+      return is_set_json_conf();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ComponentCommon)
+      return this.equals((ComponentCommon)that);
+    return false;
+  }
+
+  public boolean equals(ComponentCommon that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_inputs = true && this.is_set_inputs();
+    boolean that_present_inputs = true && that.is_set_inputs();
+    if (this_present_inputs || that_present_inputs) {
+      if (!(this_present_inputs && that_present_inputs))
+        return false;
+      if (!this.inputs.equals(that.inputs))
+        return false;
+    }
+
+    boolean this_present_streams = true && this.is_set_streams();
+    boolean that_present_streams = true && that.is_set_streams();
+    if (this_present_streams || that_present_streams) {
+      if (!(this_present_streams && that_present_streams))
+        return false;
+      if (!this.streams.equals(that.streams))
+        return false;
+    }
+
+    boolean this_present_parallelism_hint = true && this.is_set_parallelism_hint();
+    boolean that_present_parallelism_hint = true && that.is_set_parallelism_hint();
+    if (this_present_parallelism_hint || that_present_parallelism_hint) {
+      if (!(this_present_parallelism_hint && that_present_parallelism_hint))
+        return false;
+      if (this.parallelism_hint != that.parallelism_hint)
+        return false;
+    }
+
+    boolean this_present_json_conf = true && this.is_set_json_conf();
+    boolean that_present_json_conf = true && that.is_set_json_conf();
+    if (this_present_json_conf || that_present_json_conf) {
+      if (!(this_present_json_conf && that_present_json_conf))
+        return false;
+      if (!this.json_conf.equals(that.json_conf))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_inputs = true && (is_set_inputs());
+    list.add(present_inputs);
+    if (present_inputs)
+      list.add(inputs);
+
+    boolean present_streams = true && (is_set_streams());
+    list.add(present_streams);
+    if (present_streams)
+      list.add(streams);
+
+    boolean present_parallelism_hint = true && (is_set_parallelism_hint());
+    list.add(present_parallelism_hint);
+    if (present_parallelism_hint)
+      list.add(parallelism_hint);
+
+    boolean present_json_conf = true && (is_set_json_conf());
+    list.add(present_json_conf);
+    if (present_json_conf)
+      list.add(json_conf);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ComponentCommon other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_inputs()).compareTo(other.is_set_inputs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_inputs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.inputs, other.inputs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_streams()).compareTo(other.is_set_streams());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_streams()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streams, other.streams);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_parallelism_hint()).compareTo(other.is_set_parallelism_hint());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_parallelism_hint()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parallelism_hint, other.parallelism_hint);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_json_conf()).compareTo(other.is_set_json_conf());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_json_conf()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.json_conf, other.json_conf);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ComponentCommon(");
+    boolean first = true;
+
+    sb.append("inputs:");
+    if (this.inputs == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.inputs);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("streams:");
+    if (this.streams == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.streams);
+    }
+    first = false;
+    if (is_set_parallelism_hint()) {
+      if (!first) sb.append(", ");
+      sb.append("parallelism_hint:");
+      sb.append(this.parallelism_hint);
+      first = false;
+    }
+    if (is_set_json_conf()) {
+      if (!first) sb.append(", ");
+      sb.append("json_conf:");
+      if (this.json_conf == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.json_conf);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_inputs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'inputs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_streams()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'streams' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ComponentCommonStandardSchemeFactory implements SchemeFactory {
+    public ComponentCommonStandardScheme getScheme() {
+      return new ComponentCommonStandardScheme();
+    }
+  }
+
+  private static class ComponentCommonStandardScheme extends StandardScheme<ComponentCommon> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentCommon struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // INPUTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin();
+                struct.inputs = new HashMap<GlobalStreamId,Grouping>(2*_map24.size);
+                GlobalStreamId _key25;
+                Grouping _val26;
+                for (int _i27 = 0; _i27 < _map24.size; ++_i27)
+                {
+                  _key25 = new GlobalStreamId();
+                  _key25.read(iprot);
+                  _val26 = new Grouping();
+                  _val26.read(iprot);
+                  struct.inputs.put(_key25, _val26);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_inputs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STREAMS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map28 = iprot.readMapBegin();
+                struct.streams = new HashMap<String,StreamInfo>(2*_map28.size);
+                String _key29;
+                StreamInfo _val30;
+                for (int _i31 = 0; _i31 < _map28.size; ++_i31)
+                {
+                  _key29 = iprot.readString();
+                  _val30 = new StreamInfo();
+                  _val30.read(iprot);
+                  struct.streams.put(_key29, _val30);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_streams_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // PARALLELISM_HINT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.parallelism_hint = iprot.readI32();
+              struct.set_parallelism_hint_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // JSON_CONF
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.json_conf = iprot.readString();
+              struct.set_json_conf_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentCommon struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.inputs != null) {
+        oprot.writeFieldBegin(INPUTS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.inputs.size()));
+          for (Map.Entry<GlobalStreamId, Grouping> _iter32 : struct.inputs.entrySet())
+          {
+            _iter32.getKey().write(oprot);
+            _iter32.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.streams != null) {
+        oprot.writeFieldBegin(STREAMS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.streams.size()));
+          for (Map.Entry<String, StreamInfo> _iter33 : struct.streams.entrySet())
+          {
+            oprot.writeString(_iter33.getKey());
+            _iter33.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_parallelism_hint()) {
+        oprot.writeFieldBegin(PARALLELISM_HINT_FIELD_DESC);
+        oprot.writeI32(struct.parallelism_hint);
+        oprot.writeFieldEnd();
+      }
+      if (struct.json_conf != null) {
+        if (struct.is_set_json_conf()) {
+          oprot.writeFieldBegin(JSON_CONF_FIELD_DESC);
+          oprot.writeString(struct.json_conf);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ComponentCommonTupleSchemeFactory implements SchemeFactory {
+    public ComponentCommonTupleScheme getScheme() {
+      return new ComponentCommonTupleScheme();
+    }
+  }
+
+  private static class ComponentCommonTupleScheme extends TupleScheme<ComponentCommon> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ComponentCommon struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.inputs.size());
+        for (Map.Entry<GlobalStreamId, Grouping> _iter34 : struct.inputs.entrySet())
+        {
+          _iter34.getKey().write(oprot);
+          _iter34.getValue().write(oprot);
+        }
+      }
+      {
+        oprot.writeI32(struct.streams.size());
+        for (Map.Entry<String, StreamInfo> _iter35 : struct.streams.entrySet())
+        {
+          oprot.writeString(_iter35.getKey());
+          _iter35.getValue().write(oprot);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_parallelism_hint()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_json_conf()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_parallelism_hint()) {
+        oprot.writeI32(struct.parallelism_hint);
+      }
+      if (struct.is_set_json_conf()) {
+        oprot.writeString(struct.json_conf);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ComponentCommon struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TMap _map36 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.inputs = new HashMap<GlobalStreamId,Grouping>(2*_map36.size);
+        GlobalStreamId _key37;
+        Grouping _val38;
+        for (int _i39 = 0; _i39 < _map36.size; ++_i39)
+        {
+          _key37 = new GlobalStreamId();
+          _key37.read(iprot);
+          _val38 = new Grouping();
+          _val38.read(iprot);
+          struct.inputs.put(_key37, _val38);
+        }
+      }
+      struct.set_inputs_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map40 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.streams = new HashMap<String,StreamInfo>(2*_map40.size);
+        String _key41;
+        StreamInfo _val42;
+        for (int _i43 = 0; _i43 < _map40.size; ++_i43)
+        {
+          _key41 = iprot.readString();
+          _val42 = new StreamInfo();
+          _val42.read(iprot);
+          struct.streams.put(_key41, _val42);
+        }
+      }
+      struct.set_streams_isSet(true);
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.parallelism_hint = iprot.readI32();
+        struct.set_parallelism_hint_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.json_conf = iprot.readString();
+        struct.set_json_conf_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/ComponentObject.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ComponentObject.java b/storm-client/src/jvm/org/apache/storm/generated/ComponentObject.java
new file mode 100644
index 0000000..94560a0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/ComponentObject.java
@@ -0,0 +1,462 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class ComponentObject extends org.apache.thrift.TUnion<ComponentObject, ComponentObject._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentObject");
+  private static final org.apache.thrift.protocol.TField SERIALIZED_JAVA_FIELD_DESC = new org.apache.thrift.protocol.TField("serialized_java", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField SHELL_FIELD_DESC = new org.apache.thrift.protocol.TField("shell", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField JAVA_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("java_object", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SERIALIZED_JAVA((short)1, "serialized_java"),
+    SHELL((short)2, "shell"),
+    JAVA_OBJECT((short)3, "java_object");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SERIALIZED_JAVA
+          return SERIALIZED_JAVA;
+        case 2: // SHELL
+          return SHELL;
+        case 3: // JAVA_OBJECT
+          return JAVA_OBJECT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SERIALIZED_JAVA, new org.apache.thrift.meta_data.FieldMetaData("serialized_java", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.SHELL, new org.apache.thrift.meta_data.FieldMetaData("shell", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShellComponent.class)));
+    tmpMap.put(_Fields.JAVA_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("java_object", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, JavaObject.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentObject.class, metaDataMap);
+  }
+
+  public ComponentObject() {
+    super();
+  }
+
+  public ComponentObject(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public ComponentObject(ComponentObject other) {
+    super(other);
+  }
+  public ComponentObject deepCopy() {
+    return new ComponentObject(this);
+  }
+
+  public static ComponentObject serialized_java(ByteBuffer value) {
+    ComponentObject x = new ComponentObject();
+    x.set_serialized_java(value);
+    return x;
+  }
+
+  public static ComponentObject serialized_java(byte[] value) {
+    ComponentObject x = new ComponentObject();
+    x.set_serialized_java(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+    return x;
+  }
+
+  public static ComponentObject shell(ShellComponent value) {
+    ComponentObject x = new ComponentObject();
+    x.set_shell(value);
+    return x;
+  }
+
+  public static ComponentObject java_object(JavaObject value) {
+    ComponentObject x = new ComponentObject();
+    x.set_java_object(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case SERIALIZED_JAVA:
+        if (value instanceof ByteBuffer) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type ByteBuffer for field 'serialized_java', but got " + value.getClass().getSimpleName());
+      case SHELL:
+        if (value instanceof ShellComponent) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type ShellComponent for field 'shell', but got " + value.getClass().getSimpleName());
+      case JAVA_OBJECT:
+        if (value instanceof JavaObject) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type JavaObject for field 'java_object', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case SERIALIZED_JAVA:
+          if (field.type == SERIALIZED_JAVA_FIELD_DESC.type) {
+            ByteBuffer serialized_java;
+            serialized_java = iprot.readBinary();
+            return serialized_java;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SHELL:
+          if (field.type == SHELL_FIELD_DESC.type) {
+            ShellComponent shell;
+            shell = new ShellComponent();
+            shell.read(iprot);
+            return shell;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case JAVA_OBJECT:
+          if (field.type == JAVA_OBJECT_FIELD_DESC.type) {
+            JavaObject java_object;
+            java_object = new JavaObject();
+            java_object.read(iprot);
+            return java_object;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case SERIALIZED_JAVA:
+        ByteBuffer serialized_java = (ByteBuffer)value_;
+        oprot.writeBinary(serialized_java);
+        return;
+      case SHELL:
+        ShellComponent shell = (ShellComponent)value_;
+        shell.write(oprot);
+        return;
+      case JAVA_OBJECT:
+        JavaObject java_object = (JavaObject)value_;
+        java_object.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case SERIALIZED_JAVA:
+          ByteBuffer serialized_java;
+          serialized_java = iprot.readBinary();
+          return serialized_java;
+        case SHELL:
+          ShellComponent shell;
+          shell = new ShellComponent();
+          shell.read(iprot);
+          return shell;
+        case JAVA_OBJECT:
+          JavaObject java_object;
+          java_object = new JavaObject();
+          java_object.read(iprot);
+          return java_object;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case SERIALIZED_JAVA:
+        ByteBuffer serialized_java = (ByteBuffer)value_;
+        oprot.writeBinary(serialized_java);
+        return;
+      case SHELL:
+        ShellComponent shell = (ShellComponent)value_;
+        shell.write(oprot);
+        return;
+      case JAVA_OBJECT:
+        JavaObject java_object = (JavaObject)value_;
+        java_object.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case SERIALIZED_JAVA:
+        return SERIALIZED_JAVA_FIELD_DESC;
+      case SHELL:
+        return SHELL_FIELD_DESC;
+      case JAVA_OBJECT:
+        return JAVA_OBJECT_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public byte[] get_serialized_java() {
+    set_serialized_java(org.apache.thrift.TBaseHelper.rightSize(buffer_for_serialized_java()));
+    ByteBuffer b = buffer_for_serialized_java();
+    return b == null ? null : b.array();
+  }
+
+  public ByteBuffer buffer_for_serialized_java() {
+    if (getSetField() == _Fields.SERIALIZED_JAVA) {
+      return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue());
+    } else {
+      throw new RuntimeException("Cannot get field 'serialized_java' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_serialized_java(byte[] value) {
+    set_serialized_java(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+  }
+
+  public void set_serialized_java(ByteBuffer value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SERIALIZED_JAVA;
+    value_ = value;
+  }
+
+  public ShellComponent get_shell() {
+    if (getSetField() == _Fields.SHELL) {
+      return (ShellComponent)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'shell' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_shell(ShellComponent value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SHELL;
+    value_ = value;
+  }
+
+  public JavaObject get_java_object() {
+    if (getSetField() == _Fields.JAVA_OBJECT) {
+      return (JavaObject)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'java_object' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_java_object(JavaObject value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.JAVA_OBJECT;
+    value_ = value;
+  }
+
+  public boolean is_set_serialized_java() {
+    return setField_ == _Fields.SERIALIZED_JAVA;
+  }
+
+
+  public boolean is_set_shell() {
+    return setField_ == _Fields.SHELL;
+  }
+
+
+  public boolean is_set_java_object() {
+    return setField_ == _Fields.JAVA_OBJECT;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof ComponentObject) {
+      return equals((ComponentObject)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(ComponentObject other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(ComponentObject other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}


[08/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/generated/TopologyPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyPageInfo.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyPageInfo.java
new file mode 100644
index 0000000..a0e25ce
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyPageInfo.java
@@ -0,0 +1,2757 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInfo, TopologyPageInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyPageInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyPageInfo");
+
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)7);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_conf", org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField ID_TO_SPOUT_AGG_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("id_to_spout_agg_stats", org.apache.thrift.protocol.TType.MAP, (short)9);
+  private static final org.apache.thrift.protocol.TField ID_TO_BOLT_AGG_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("id_to_bolt_agg_stats", org.apache.thrift.protocol.TType.MAP, (short)10);
+  private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)11);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_stats", org.apache.thrift.protocol.TType.STRUCT, (short)12);
+  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)13);
+  private static final org.apache.thrift.protocol.TField DEBUG_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("debug_options", org.apache.thrift.protocol.TType.STRUCT, (short)14);
+  private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)15);
+  private static final org.apache.thrift.protocol.TField WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("workers", org.apache.thrift.protocol.TType.LIST, (short)16);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologyPageInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologyPageInfoTupleSchemeFactory());
+  }
+
+  private String id; // required
+  private String name; // optional
+  private int uptime_secs; // optional
+  private String status; // optional
+  private int num_tasks; // optional
+  private int num_workers; // optional
+  private int num_executors; // optional
+  private String topology_conf; // optional
+  private Map<String,ComponentAggregateStats> id_to_spout_agg_stats; // optional
+  private Map<String,ComponentAggregateStats> id_to_bolt_agg_stats; // optional
+  private String sched_status; // optional
+  private TopologyStats topology_stats; // optional
+  private String owner; // optional
+  private DebugOptions debug_options; // optional
+  private int replication_count; // optional
+  private List<WorkerSummary> workers; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ID((short)1, "id"),
+    NAME((short)2, "name"),
+    UPTIME_SECS((short)3, "uptime_secs"),
+    STATUS((short)4, "status"),
+    NUM_TASKS((short)5, "num_tasks"),
+    NUM_WORKERS((short)6, "num_workers"),
+    NUM_EXECUTORS((short)7, "num_executors"),
+    TOPOLOGY_CONF((short)8, "topology_conf"),
+    ID_TO_SPOUT_AGG_STATS((short)9, "id_to_spout_agg_stats"),
+    ID_TO_BOLT_AGG_STATS((short)10, "id_to_bolt_agg_stats"),
+    SCHED_STATUS((short)11, "sched_status"),
+    TOPOLOGY_STATS((short)12, "topology_stats"),
+    OWNER((short)13, "owner"),
+    DEBUG_OPTIONS((short)14, "debug_options"),
+    REPLICATION_COUNT((short)15, "replication_count"),
+    WORKERS((short)16, "workers"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // NAME
+          return NAME;
+        case 3: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 4: // STATUS
+          return STATUS;
+        case 5: // NUM_TASKS
+          return NUM_TASKS;
+        case 6: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 7: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 8: // TOPOLOGY_CONF
+          return TOPOLOGY_CONF;
+        case 9: // ID_TO_SPOUT_AGG_STATS
+          return ID_TO_SPOUT_AGG_STATS;
+        case 10: // ID_TO_BOLT_AGG_STATS
+          return ID_TO_BOLT_AGG_STATS;
+        case 11: // SCHED_STATUS
+          return SCHED_STATUS;
+        case 12: // TOPOLOGY_STATS
+          return TOPOLOGY_STATS;
+        case 13: // OWNER
+          return OWNER;
+        case 14: // DEBUG_OPTIONS
+          return DEBUG_OPTIONS;
+        case 15: // REPLICATION_COUNT
+          return REPLICATION_COUNT;
+        case 16: // WORKERS
+          return WORKERS;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __UPTIME_SECS_ISSET_ID = 0;
+  private static final int __NUM_TASKS_ISSET_ID = 1;
+  private static final int __NUM_WORKERS_ISSET_ID = 2;
+  private static final int __NUM_EXECUTORS_ISSET_ID = 3;
+  private static final int __REPLICATION_COUNT_ISSET_ID = 4;
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __REQUESTED_CPU_ISSET_ID = 7;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 8;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 9;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 10;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NAME,_Fields.UPTIME_SECS,_Fields.STATUS,_Fields.NUM_TASKS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS,_Fields.TOPOLOGY_CONF,_Fields.ID_TO_SPOUT_AGG_STATS,_Fields.ID_TO_BOLT_AGG_STATS,_Fields.SCHED_STATUS,_Fields.TOPOLOGY_STATS,_Fields.OWNER,_Fields.DEBUG_OPTIONS,_Fields.REPLICATION_COUNT,_Fields.WORKERS,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.TOPOLOGY_CONF, new org.apache.thrift.meta_data.FieldMetaData("topology_conf", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ID_TO_SPOUT_AGG_STATS, new org.apache.thrift.meta_data.FieldMetaData("id_to_spout_agg_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.ID_TO_BOLT_AGG_STATS, new org.apache.thrift.meta_data.FieldMetaData("id_to_bolt_agg_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOPOLOGY_STATS, new org.apache.thrift.meta_data.FieldMetaData("topology_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyStats.class)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DEBUG_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("debug_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class)));
+    tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.WORKERS, new org.apache.thrift.meta_data.FieldMetaData("workers", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerSummary.class))));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyPageInfo.class, metaDataMap);
+  }
+
+  public TopologyPageInfo() {
+  }
+
+  public TopologyPageInfo(
+    String id)
+  {
+    this();
+    this.id = id;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyPageInfo(TopologyPageInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_status()) {
+      this.status = other.status;
+    }
+    this.num_tasks = other.num_tasks;
+    this.num_workers = other.num_workers;
+    this.num_executors = other.num_executors;
+    if (other.is_set_topology_conf()) {
+      this.topology_conf = other.topology_conf;
+    }
+    if (other.is_set_id_to_spout_agg_stats()) {
+      Map<String,ComponentAggregateStats> __this__id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(other.id_to_spout_agg_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.id_to_spout_agg_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__id_to_spout_agg_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__id_to_spout_agg_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__id_to_spout_agg_stats.put(__this__id_to_spout_agg_stats_copy_key, __this__id_to_spout_agg_stats_copy_value);
+      }
+      this.id_to_spout_agg_stats = __this__id_to_spout_agg_stats;
+    }
+    if (other.is_set_id_to_bolt_agg_stats()) {
+      Map<String,ComponentAggregateStats> __this__id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(other.id_to_bolt_agg_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.id_to_bolt_agg_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__id_to_bolt_agg_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__id_to_bolt_agg_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__id_to_bolt_agg_stats.put(__this__id_to_bolt_agg_stats_copy_key, __this__id_to_bolt_agg_stats_copy_value);
+      }
+      this.id_to_bolt_agg_stats = __this__id_to_bolt_agg_stats;
+    }
+    if (other.is_set_sched_status()) {
+      this.sched_status = other.sched_status;
+    }
+    if (other.is_set_topology_stats()) {
+      this.topology_stats = new TopologyStats(other.topology_stats);
+    }
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
+    if (other.is_set_debug_options()) {
+      this.debug_options = new DebugOptions(other.debug_options);
+    }
+    this.replication_count = other.replication_count;
+    if (other.is_set_workers()) {
+      List<WorkerSummary> __this__workers = new ArrayList<WorkerSummary>(other.workers.size());
+      for (WorkerSummary other_element : other.workers) {
+        __this__workers.add(new WorkerSummary(other_element));
+      }
+      this.workers = __this__workers;
+    }
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
+  }
+
+  public TopologyPageInfo deepCopy() {
+    return new TopologyPageInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.id = null;
+    this.name = null;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    this.status = null;
+    set_num_tasks_isSet(false);
+    this.num_tasks = 0;
+    set_num_workers_isSet(false);
+    this.num_workers = 0;
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    this.topology_conf = null;
+    this.id_to_spout_agg_stats = null;
+    this.id_to_bolt_agg_stats = null;
+    this.sched_status = null;
+    this.topology_stats = null;
+    this.owner = null;
+    this.debug_options = null;
+    set_replication_count_isSet(false);
+    this.replication_count = 0;
+    this.workers = null;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_status() {
+    return this.status;
+  }
+
+  public void set_status(String status) {
+    this.status = status;
+  }
+
+  public void unset_status() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_status() {
+    return this.status != null;
+  }
+
+  public void set_status_isSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value);
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public String get_topology_conf() {
+    return this.topology_conf;
+  }
+
+  public void set_topology_conf(String topology_conf) {
+    this.topology_conf = topology_conf;
+  }
+
+  public void unset_topology_conf() {
+    this.topology_conf = null;
+  }
+
+  /** Returns true if field topology_conf is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_conf() {
+    return this.topology_conf != null;
+  }
+
+  public void set_topology_conf_isSet(boolean value) {
+    if (!value) {
+      this.topology_conf = null;
+    }
+  }
+
+  public int get_id_to_spout_agg_stats_size() {
+    return (this.id_to_spout_agg_stats == null) ? 0 : this.id_to_spout_agg_stats.size();
+  }
+
+  public void put_to_id_to_spout_agg_stats(String key, ComponentAggregateStats val) {
+    if (this.id_to_spout_agg_stats == null) {
+      this.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.id_to_spout_agg_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_id_to_spout_agg_stats() {
+    return this.id_to_spout_agg_stats;
+  }
+
+  public void set_id_to_spout_agg_stats(Map<String,ComponentAggregateStats> id_to_spout_agg_stats) {
+    this.id_to_spout_agg_stats = id_to_spout_agg_stats;
+  }
+
+  public void unset_id_to_spout_agg_stats() {
+    this.id_to_spout_agg_stats = null;
+  }
+
+  /** Returns true if field id_to_spout_agg_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id_to_spout_agg_stats() {
+    return this.id_to_spout_agg_stats != null;
+  }
+
+  public void set_id_to_spout_agg_stats_isSet(boolean value) {
+    if (!value) {
+      this.id_to_spout_agg_stats = null;
+    }
+  }
+
+  public int get_id_to_bolt_agg_stats_size() {
+    return (this.id_to_bolt_agg_stats == null) ? 0 : this.id_to_bolt_agg_stats.size();
+  }
+
+  public void put_to_id_to_bolt_agg_stats(String key, ComponentAggregateStats val) {
+    if (this.id_to_bolt_agg_stats == null) {
+      this.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.id_to_bolt_agg_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_id_to_bolt_agg_stats() {
+    return this.id_to_bolt_agg_stats;
+  }
+
+  public void set_id_to_bolt_agg_stats(Map<String,ComponentAggregateStats> id_to_bolt_agg_stats) {
+    this.id_to_bolt_agg_stats = id_to_bolt_agg_stats;
+  }
+
+  public void unset_id_to_bolt_agg_stats() {
+    this.id_to_bolt_agg_stats = null;
+  }
+
+  /** Returns true if field id_to_bolt_agg_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id_to_bolt_agg_stats() {
+    return this.id_to_bolt_agg_stats != null;
+  }
+
+  public void set_id_to_bolt_agg_stats_isSet(boolean value) {
+    if (!value) {
+      this.id_to_bolt_agg_stats = null;
+    }
+  }
+
+  public String get_sched_status() {
+    return this.sched_status;
+  }
+
+  public void set_sched_status(String sched_status) {
+    this.sched_status = sched_status;
+  }
+
+  public void unset_sched_status() {
+    this.sched_status = null;
+  }
+
+  /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sched_status() {
+    return this.sched_status != null;
+  }
+
+  public void set_sched_status_isSet(boolean value) {
+    if (!value) {
+      this.sched_status = null;
+    }
+  }
+
+  public TopologyStats get_topology_stats() {
+    return this.topology_stats;
+  }
+
+  public void set_topology_stats(TopologyStats topology_stats) {
+    this.topology_stats = topology_stats;
+  }
+
+  public void unset_topology_stats() {
+    this.topology_stats = null;
+  }
+
+  /** Returns true if field topology_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_stats() {
+    return this.topology_stats != null;
+  }
+
+  public void set_topology_stats_isSet(boolean value) {
+    if (!value) {
+      this.topology_stats = null;
+    }
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = null;
+  }
+
+  /** Returns true if field owner is set (has been assigned a value) and false otherwise */
+  public boolean is_set_owner() {
+    return this.owner != null;
+  }
+
+  public void set_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
+  public DebugOptions get_debug_options() {
+    return this.debug_options;
+  }
+
+  public void set_debug_options(DebugOptions debug_options) {
+    this.debug_options = debug_options;
+  }
+
+  public void unset_debug_options() {
+    this.debug_options = null;
+  }
+
+  /** Returns true if field debug_options is set (has been assigned a value) and false otherwise */
+  public boolean is_set_debug_options() {
+    return this.debug_options != null;
+  }
+
+  public void set_debug_options_isSet(boolean value) {
+    if (!value) {
+      this.debug_options = null;
+    }
+  }
+
+  public int get_replication_count() {
+    return this.replication_count;
+  }
+
+  public void set_replication_count(int replication_count) {
+    this.replication_count = replication_count;
+    set_replication_count_isSet(true);
+  }
+
+  public void unset_replication_count() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_count() {
+    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  public void set_replication_count_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
+  }
+
+  public int get_workers_size() {
+    return (this.workers == null) ? 0 : this.workers.size();
+  }
+
+  public java.util.Iterator<WorkerSummary> get_workers_iterator() {
+    return (this.workers == null) ? null : this.workers.iterator();
+  }
+
+  public void add_to_workers(WorkerSummary elem) {
+    if (this.workers == null) {
+      this.workers = new ArrayList<WorkerSummary>();
+    }
+    this.workers.add(elem);
+  }
+
+  public List<WorkerSummary> get_workers() {
+    return this.workers;
+  }
+
+  public void set_workers(List<WorkerSummary> workers) {
+    this.workers = workers;
+  }
+
+  public void unset_workers() {
+    this.workers = null;
+  }
+
+  /** Returns true if field workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_workers() {
+    return this.workers != null;
+  }
+
+  public void set_workers_isSet(boolean value) {
+    if (!value) {
+      this.workers = null;
+    }
+  }
+
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unset_status();
+      } else {
+        set_status((String)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case TOPOLOGY_CONF:
+      if (value == null) {
+        unset_topology_conf();
+      } else {
+        set_topology_conf((String)value);
+      }
+      break;
+
+    case ID_TO_SPOUT_AGG_STATS:
+      if (value == null) {
+        unset_id_to_spout_agg_stats();
+      } else {
+        set_id_to_spout_agg_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case ID_TO_BOLT_AGG_STATS:
+      if (value == null) {
+        unset_id_to_bolt_agg_stats();
+      } else {
+        set_id_to_bolt_agg_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case SCHED_STATUS:
+      if (value == null) {
+        unset_sched_status();
+      } else {
+        set_sched_status((String)value);
+      }
+      break;
+
+    case TOPOLOGY_STATS:
+      if (value == null) {
+        unset_topology_stats();
+      } else {
+        set_topology_stats((TopologyStats)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
+    case DEBUG_OPTIONS:
+      if (value == null) {
+        unset_debug_options();
+      } else {
+        set_debug_options((DebugOptions)value);
+      }
+      break;
+
+    case REPLICATION_COUNT:
+      if (value == null) {
+        unset_replication_count();
+      } else {
+        set_replication_count((Integer)value);
+      }
+      break;
+
+    case WORKERS:
+      if (value == null) {
+        unset_workers();
+      } else {
+        set_workers((List<WorkerSummary>)value);
+      }
+      break;
+
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case NAME:
+      return get_name();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case STATUS:
+      return get_status();
+
+    case NUM_TASKS:
+      return get_num_tasks();
+
+    case NUM_WORKERS:
+      return get_num_workers();
+
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case TOPOLOGY_CONF:
+      return get_topology_conf();
+
+    case ID_TO_SPOUT_AGG_STATS:
+      return get_id_to_spout_agg_stats();
+
+    case ID_TO_BOLT_AGG_STATS:
+      return get_id_to_bolt_agg_stats();
+
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case TOPOLOGY_STATS:
+      return get_topology_stats();
+
+    case OWNER:
+      return get_owner();
+
+    case DEBUG_OPTIONS:
+      return get_debug_options();
+
+    case REPLICATION_COUNT:
+      return get_replication_count();
+
+    case WORKERS:
+      return get_workers();
+
+    case REQUESTED_MEMONHEAP:
+      return get_requested_memonheap();
+
+    case REQUESTED_MEMOFFHEAP:
+      return get_requested_memoffheap();
+
+    case REQUESTED_CPU:
+      return get_requested_cpu();
+
+    case ASSIGNED_MEMONHEAP:
+      return get_assigned_memonheap();
+
+    case ASSIGNED_MEMOFFHEAP:
+      return get_assigned_memoffheap();
+
+    case ASSIGNED_CPU:
+      return get_assigned_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case NAME:
+      return is_set_name();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case STATUS:
+      return is_set_status();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case TOPOLOGY_CONF:
+      return is_set_topology_conf();
+    case ID_TO_SPOUT_AGG_STATS:
+      return is_set_id_to_spout_agg_stats();
+    case ID_TO_BOLT_AGG_STATS:
+      return is_set_id_to_bolt_agg_stats();
+    case SCHED_STATUS:
+      return is_set_sched_status();
+    case TOPOLOGY_STATS:
+      return is_set_topology_stats();
+    case OWNER:
+      return is_set_owner();
+    case DEBUG_OPTIONS:
+      return is_set_debug_options();
+    case REPLICATION_COUNT:
+      return is_set_replication_count();
+    case WORKERS:
+      return is_set_workers();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyPageInfo)
+      return this.equals((TopologyPageInfo)that);
+    return false;
+  }
+
+  public boolean equals(TopologyPageInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true && this.is_set_uptime_secs();
+    boolean that_present_uptime_secs = true && that.is_set_uptime_secs();
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_status = true && this.is_set_status();
+    boolean that_present_status = true && that.is_set_status();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    boolean this_present_num_tasks = true && this.is_set_num_tasks();
+    boolean that_present_num_tasks = true && that.is_set_num_tasks();
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_num_workers = true && this.is_set_num_workers();
+    boolean that_present_num_workers = true && that.is_set_num_workers();
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_topology_conf = true && this.is_set_topology_conf();
+    boolean that_present_topology_conf = true && that.is_set_topology_conf();
+    if (this_present_topology_conf || that_present_topology_conf) {
+      if (!(this_present_topology_conf && that_present_topology_conf))
+        return false;
+      if (!this.topology_conf.equals(that.topology_conf))
+        return false;
+    }
+
+    boolean this_present_id_to_spout_agg_stats = true && this.is_set_id_to_spout_agg_stats();
+    boolean that_present_id_to_spout_agg_stats = true && that.is_set_id_to_spout_agg_stats();
+    if (this_present_id_to_spout_agg_stats || that_present_id_to_spout_agg_stats) {
+      if (!(this_present_id_to_spout_agg_stats && that_present_id_to_spout_agg_stats))
+        return false;
+      if (!this.id_to_spout_agg_stats.equals(that.id_to_spout_agg_stats))
+        return false;
+    }
+
+    boolean this_present_id_to_bolt_agg_stats = true && this.is_set_id_to_bolt_agg_stats();
+    boolean that_present_id_to_bolt_agg_stats = true && that.is_set_id_to_bolt_agg_stats();
+    if (this_present_id_to_bolt_agg_stats || that_present_id_to_bolt_agg_stats) {
+      if (!(this_present_id_to_bolt_agg_stats && that_present_id_to_bolt_agg_stats))
+        return false;
+      if (!this.id_to_bolt_agg_stats.equals(that.id_to_bolt_agg_stats))
+        return false;
+    }
+
+    boolean this_present_sched_status = true && this.is_set_sched_status();
+    boolean that_present_sched_status = true && that.is_set_sched_status();
+    if (this_present_sched_status || that_present_sched_status) {
+      if (!(this_present_sched_status && that_present_sched_status))
+        return false;
+      if (!this.sched_status.equals(that.sched_status))
+        return false;
+    }
+
+    boolean this_present_topology_stats = true && this.is_set_topology_stats();
+    boolean that_present_topology_stats = true && that.is_set_topology_stats();
+    if (this_present_topology_stats || that_present_topology_stats) {
+      if (!(this_present_topology_stats && that_present_topology_stats))
+        return false;
+      if (!this.topology_stats.equals(that.topology_stats))
+        return false;
+    }
+
+    boolean this_present_owner = true && this.is_set_owner();
+    boolean that_present_owner = true && that.is_set_owner();
+    if (this_present_owner || that_present_owner) {
+      if (!(this_present_owner && that_present_owner))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
+    boolean this_present_debug_options = true && this.is_set_debug_options();
+    boolean that_present_debug_options = true && that.is_set_debug_options();
+    if (this_present_debug_options || that_present_debug_options) {
+      if (!(this_present_debug_options && that_present_debug_options))
+        return false;
+      if (!this.debug_options.equals(that.debug_options))
+        return false;
+    }
+
+    boolean this_present_replication_count = true && this.is_set_replication_count();
+    boolean that_present_replication_count = true && that.is_set_replication_count();
+    if (this_present_replication_count || that_present_replication_count) {
+      if (!(this_present_replication_count && that_present_replication_count))
+        return false;
+      if (this.replication_count != that.replication_count)
+        return false;
+    }
+
+    boolean this_present_workers = true && this.is_set_workers();
+    boolean that_present_workers = true && that.is_set_workers();
+    if (this_present_workers || that_present_workers) {
+      if (!(this_present_workers && that_present_workers))
+        return false;
+      if (!this.workers.equals(that.workers))
+        return false;
+    }
+
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_id = true && (is_set_id());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_uptime_secs = true && (is_set_uptime_secs());
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_status = true && (is_set_status());
+    list.add(present_status);
+    if (present_status)
+      list.add(status);
+
+    boolean present_num_tasks = true && (is_set_num_tasks());
+    list.add(present_num_tasks);
+    if (present_num_tasks)
+      list.add(num_tasks);
+
+    boolean present_num_workers = true && (is_set_num_workers());
+    list.add(present_num_workers);
+    if (present_num_workers)
+      list.add(num_workers);
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_topology_conf = true && (is_set_topology_conf());
+    list.add(present_topology_conf);
+    if (present_topology_conf)
+      list.add(topology_conf);
+
+    boolean present_id_to_spout_agg_stats = true && (is_set_id_to_spout_agg_stats());
+    list.add(present_id_to_spout_agg_stats);
+    if (present_id_to_spout_agg_stats)
+      list.add(id_to_spout_agg_stats);
+
+    boolean present_id_to_bolt_agg_stats = true && (is_set_id_to_bolt_agg_stats());
+    list.add(present_id_to_bolt_agg_stats);
+    if (present_id_to_bolt_agg_stats)
+      list.add(id_to_bolt_agg_stats);
+
+    boolean present_sched_status = true && (is_set_sched_status());
+    list.add(present_sched_status);
+    if (present_sched_status)
+      list.add(sched_status);
+
+    boolean present_topology_stats = true && (is_set_topology_stats());
+    list.add(present_topology_stats);
+    if (present_topology_stats)
+      list.add(topology_stats);
+
+    boolean present_owner = true && (is_set_owner());
+    list.add(present_owner);
+    if (present_owner)
+      list.add(owner);
+
+    boolean present_debug_options = true && (is_set_debug_options());
+    list.add(present_debug_options);
+    if (present_debug_options)
+      list.add(debug_options);
+
+    boolean present_replication_count = true && (is_set_replication_count());
+    list.add(present_replication_count);
+    if (present_replication_count)
+      list.add(replication_count);
+
+    boolean present_workers = true && (is_set_workers());
+    list.add(present_workers);
+    if (present_workers)
+      list.add(workers);
+
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologyPageInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_conf()).compareTo(other.is_set_topology_conf());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_conf()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_conf, other.topology_conf);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_id_to_spout_agg_stats()).compareTo(other.is_set_id_to_spout_agg_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id_to_spout_agg_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id_to_spout_agg_stats, other.id_to_spout_agg_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_id_to_bolt_agg_stats()).compareTo(other.is_set_id_to_bolt_agg_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id_to_bolt_agg_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id_to_bolt_agg_stats, other.id_to_bolt_agg_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(other.is_set_sched_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sched_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, other.sched_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_stats()).compareTo(other.is_set_topology_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_stats, other.topology_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_debug_options()).compareTo(other.is_set_debug_options());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_debug_options()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug_options, other.debug_options);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(other.is_set_replication_count());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_count()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, other.replication_count);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_workers()).compareTo(other.is_set_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.workers, other.workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyPageInfo(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (is_set_name()) {
+      if (!first) sb.append(", ");
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+    }
+    if (is_set_uptime_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("uptime_secs:");
+      sb.append(this.uptime_secs);
+      first = false;
+    }
+    if (is_set_status()) {
+      if (!first) sb.append(", ");
+      sb.append("status:");
+      if (this.status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.status);
+      }
+      first = false;
+    }
+    if (is_set_num_tasks()) {
+      if (!first) sb.append(", ");
+      sb.append("num_tasks:");
+      sb.append(this.num_tasks);
+      first = false;
+    }
+    if (is_set_num_workers()) {
+      if (!first) sb.append(", ");
+      sb.append("num_workers:");
+      sb.append(this.num_workers);
+      first = false;
+    }
+    if (is_set_num_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("num_executors:");
+      sb.append(this.num_executors);
+      first = false;
+    }
+    if (is_set_topology_conf()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_conf:");
+      if (this.topology_conf == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_conf);
+      }
+      first = false;
+    }
+    if (is_set_id_to_spout_agg_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("id_to_spout_agg_stats:");
+      if (this.id_to_spout_agg_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id_to_spout_agg_stats);
+      }
+      first = false;
+    }
+    if (is_set_id_to_bolt_agg_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("id_to_bolt_agg_stats:");
+      if (this.id_to_bolt_agg_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id_to_bolt_agg_stats);
+      }
+      first = false;
+    }
+    if (is_set_sched_status()) {
+      if (!first) sb.append(", ");
+      sb.append("sched_status:");
+      if (this.sched_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sched_status);
+      }
+      first = false;
+    }
+    if (is_set_topology_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_stats:");
+      if (this.topology_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_stats);
+      }
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
+    if (is_set_debug_options()) {
+      if (!first) sb.append(", ");
+      sb.append("debug_options:");
+      if (this.debug_options == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.debug_options);
+      }
+      first = false;
+    }
+    if (is_set_replication_count()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_count:");
+      sb.append(this.replication_count);
+      first = false;
+    }
+    if (is_set_workers()) {
+      if (!first) sb.append(", ");
+      sb.append("workers:");
+      if (this.workers == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.workers);
+      }
+      first = false;
+    }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (topology_stats != null) {
+      topology_stats.validate();
+    }
+    if (debug_options != null) {
+      debug_options.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologyPageInfoStandardSchemeFactory implements SchemeFactory {
+    public TopologyPageInfoStandardScheme getScheme() {
+      return new TopologyPageInfoStandardScheme();
+    }
+  }
+
+  private static class TopologyPageInfoStandardScheme extends StandardScheme<TopologyPageInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyPageInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.id = iprot.readString();
+              struct.set_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.status = iprot.readString();
+              struct.set_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_tasks = iprot.readI32();
+              struct.set_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // NUM_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_workers = iprot.readI32();
+              struct.set_num_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // TOPOLOGY_CONF
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_conf = iprot.readString();
+              struct.set_topology_conf_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // ID_TO_SPOUT_AGG_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map454 = iprot.readMapBegin();
+                struct.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map454.size);
+                String _key455;
+                ComponentAggregateStats _val456;
+                for (int _i457 = 0; _i457 < _map454.size; ++_i457)
+                {
+                  _key455 = iprot.readString();
+                  _val456 = new ComponentAggregateStats();
+                  _val456.read(iprot);
+                  struct.id_to_spout_agg_stats.put(_key455, _val456);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_id_to_spout_agg_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // ID_TO_BOLT_AGG_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map458 = iprot.readMapBegin();
+                struct.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map458.size);
+                String _key459;
+                ComponentAggregateStats _val460;
+                for (int _i461 = 0; _i461 < _map458.size; ++_i461)
+                {
+                  _key459 = iprot.readString();
+                  _val460 = new ComponentAggregateStats();
+                  _val460.read(iprot);
+                  struct.id_to_bolt_agg_stats.put(_key459, _val460);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_id_to_bolt_agg_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 11: // SCHED_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.sched_status = iprot.readString();
+              struct.set_sched_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // TOPOLOGY_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.topology_stats = new TopologyStats();
+              struct.topology_stats.read(iprot);
+              struct.set_topology_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // OWNER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.owner = iprot.readString();
+              struct.set_owner_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 14: // DEBUG_OPTIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.debug_options = new DebugOptions();
+              struct.debug_options.read(iprot);
+              struct.set_debug_options_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 15: // REPLICATION_COUNT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.replication_count = iprot.readI32();
+              struct.set_replication_count_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 16: // WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list462 = iprot.readListBegin();
+                struct.workers = new ArrayList<WorkerSummary>(_list462.size);
+                WorkerSummary _elem463;
+                for (int _i464 = 0; _i464 < _list462.size; ++_i464)
+                {
+                  _elem463 = new WorkerSummary();
+                  _elem463.read(iprot);
+                  struct.workers.add(_elem463);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyPageInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(struct.id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.name != null) {
+        if (struct.is_set_name()) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.uptime_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.status != null) {
+        if (struct.is_set_status()) {
+          oprot.writeFieldBegin(STATUS_FIELD_DESC);
+          oprot.writeString(struct.status);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+        oprot.writeI32(struct.num_tasks);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_workers()) {
+        oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+        oprot.writeI32(struct.num_workers);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+        oprot.writeI32(struct.num_executors);
+        oprot.writeFieldEnd();
+      }
+      if (struct.topology_conf != null) {
+        if (struct.is_set_topology

<TRUNCATED>

[43/52] [partial] storm git commit: STORM-2441 Break down 'storm-core' to extract client (worker) artifacts

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
new file mode 100644
index 0000000..b255dad
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.*;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.storm.Config;
+import org.apache.storm.callback.DefaultWatcherCallBack;
+import org.apache.storm.callback.WatcherCallBack;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.zookeeper.ClientZookeeper;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class ZKStateStorage implements IStateStorage {
+
+    private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class);
+
+    private ConcurrentHashMap<String, ZKStateChangedCallback> callbacks = new ConcurrentHashMap<String, ZKStateChangedCallback>();
+    private CuratorFramework zkWriter;
+    private CuratorFramework zkReader;
+    private AtomicBoolean active;
+
+    private boolean isNimbus;
+    private Map authConf;
+    private Map<Object, Object> conf;
+
+    private class ZkWatcherCallBack implements WatcherCallBack{
+        @Override
+        public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+            if (active.get()) {
+                if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+                    LOG.debug("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+                } else {
+                    LOG.debug("Received event {} : {} : {}", state, type, path);
+                }
+
+                if (!type.equals(Watcher.Event.EventType.None)) {
+                    for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
+                        ZKStateChangedCallback fn = e.getValue();
+                        fn.changed(type, path);
+                    }
+                }
+            }
+        }
+    }
+
+    public ZKStateStorage(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
+        this.conf = conf;
+        this.authConf = authConf;
+        if (context.getDaemonType().equals(DaemonType.NIMBUS))
+            this.isNimbus = true;
+
+        // just mkdir STORM_ZOOKEEPER_ROOT dir
+        CuratorFramework zkTemp = mkZk();
+        String rootPath = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
+        ClientZookeeper.mkdirs(zkTemp, rootPath, acls);
+        zkTemp.close();
+
+        active = new AtomicBoolean(true);
+        zkWriter = mkZk(new ZkWatcherCallBack());
+        if (isNimbus) {
+            zkReader = mkZk(new ZkWatcherCallBack());
+        } else {
+            zkReader = zkWriter;
+        }
+
+    }
+
+    @SuppressWarnings("unchecked")
+    private CuratorFramework mkZk() throws IOException {
+        return ClientZookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "",
+                new DefaultWatcherCallBack(), authConf);
+    }
+
+    @SuppressWarnings("unchecked")
+    private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatException, IOException {
+        return ClientZookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT),
+                String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher, authConf);
+    }
+
+    @Override
+    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+        ClientZookeeper.deleteNodeBlobstore(zkWriter, path, nimbusHostPortInfo);
+    }
+
+    @Override
+    public String register(ZKStateChangedCallback callback) {
+        String id = UUID.randomUUID().toString();
+        this.callbacks.put(id, callback);
+        return id;
+    }
+
+    @Override
+    public void unregister(String id) {
+        this.callbacks.remove(id);
+    }
+
+    @Override
+    public String create_sequential(String path, byte[] data, List<ACL> acls) {
+        return ClientZookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls);
+    }
+
+    @Override
+    public void mkdirs(String path, List<ACL> acls) {
+        ClientZookeeper.mkdirs(zkWriter, path, acls);
+    }
+
+    @Override
+    public void delete_node(String path) {
+        ClientZookeeper.deleteNode(zkWriter, path);
+    }
+
+    @Override
+    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+        ClientZookeeper.mkdirs(zkWriter, ClientZookeeper.parentPath(path), acls);
+        if (ClientZookeeper.exists(zkWriter, path, false)) {
+            try {
+                ClientZookeeper.setData(zkWriter, path, data);
+            } catch (RuntimeException e) {
+                if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
+                    ClientZookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+                } else {
+                    throw e;
+                }
+            }
+
+        } else {
+            ClientZookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+        }
+    }
+
+    @Override
+    public Integer get_version(String path, boolean watch) throws Exception {
+        Integer ret = ClientZookeeper.getVersion(zkReader, path, watch);
+        return ret;
+    }
+
+    @Override
+    public boolean node_exists(String path, boolean watch) {
+        return ClientZookeeper.existsNode(zkReader, path, watch);
+    }
+
+    @Override
+    public List<String> get_children(String path, boolean watch) {
+        return ClientZookeeper.getChildren(zkReader, path, watch);
+    }
+
+    @Override
+    public void close() {
+        this.active.set(false);
+        zkWriter.close();
+        if (isNimbus) {
+            zkReader.close();
+        }
+    }
+
+    @Override
+    public void set_data(String path, byte[] data, List<ACL> acls) {
+        if (ClientZookeeper.exists(zkWriter, path, false)) {
+            ClientZookeeper.setData(zkWriter, path, data);
+        } else {
+            ClientZookeeper.mkdirs(zkWriter, ClientZookeeper.parentPath(path), acls);
+            try {
+                ClientZookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
+            } catch (RuntimeException e) {
+                if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
+                    ClientZookeeper.setData(zkWriter, path, data);
+                } else {
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Override
+    public byte[] get_data(String path, boolean watch) {
+        byte[] ret = null;
+
+        ret = ClientZookeeper.getData(zkReader, path, watch);
+
+        return ret;
+    }
+
+    @Override
+    public VersionedData<byte[]> get_data_with_version(String path, boolean watch) {
+        return ClientZookeeper.getDataWithVersion(zkReader, path, watch);
+    }
+
+    @Override
+    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+        set_data(path, data, acls);
+    }
+
+    @Override
+    public byte[] get_worker_hb(String path, boolean watch) {
+        return ClientZookeeper.getData(zkReader, path, watch);
+    }
+
+    @Override
+    public List<String> get_worker_hb_children(String path, boolean watch) {
+        return get_children(path, watch);
+    }
+
+    @Override
+    public void delete_worker_hb(String path) {
+        delete_node(path);
+    }
+
+    @Override
+    public void add_listener(final ConnectionStateListener listener) {
+        ClientZookeeper.addListener(zkReader, new ConnectionStateListener() {
+            @Override
+            public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
+                listener.stateChanged(curatorFramework, connectionState);
+            }
+        });
+    }
+
+    @Override
+    public void sync_path(String path) {
+        ClientZookeeper.syncPath(zkWriter, path);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
new file mode 100644
index 0000000..3715e48
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.cluster;
+
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+
+import java.util.List;
+import java.util.Map;
+
+public class ZKStateStorageFactory implements StateStorageFactory {
+
+    @Override
+    public IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) {
+        try {
+            return new ZKStateStorage(config, auth_conf, acls, context);
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java
new file mode 100644
index 0000000..ba6417f
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java
@@ -0,0 +1,216 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class CgroupCenter implements CgroupOperation {
+
+    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
+
+    private static CgroupCenter instance;
+
+    private CgroupCenter() {
+
+    }
+
+    public synchronized static CgroupCenter getInstance() {
+        if (CgroupUtils.enabled()) {
+            instance = new CgroupCenter();
+            return instance;
+        }
+        return null;
+    }
+
+    @Override
+    public List<Hierarchy> getHierarchies() {
+        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
+        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
+             BufferedReader br = new BufferedReader(reader)) {
+            String str = null;
+            while ((str = br.readLine()) != null) {
+                String[] strSplit = str.split(" ");
+                if (!strSplit[2].equals("cgroup")) {
+                    continue;
+                }
+                String name = strSplit[0];
+                String type = strSplit[3];
+                String dir = strSplit[1];
+                //Some mount options (i.e. rw and relatime) in type are not cgroups related
+                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
+                hierarchies.put(type, h);
+            }
+            return new ArrayList<Hierarchy>(hierarchies.values());
+        } catch (Exception e) {
+            LOG.error("Get hierarchies error {}", e);
+        }
+        return null;
+    }
+
+    @Override
+    public Set<SubSystem> getSubSystems() {
+        Set<SubSystem> subSystems = new HashSet<SubSystem>();
+        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
+             BufferedReader br = new BufferedReader(reader)){
+            String str = null;
+            while ((str = br.readLine()) != null) {
+                String[] split = str.split("\t");
+                SubSystemType type = SubSystemType.getSubSystem(split[0]);
+                if (type == null) {
+                    continue;
+                }
+                int hierarchyID = Integer.valueOf(split[1]);
+                int cgroupNum = Integer.valueOf(split[2]);
+                boolean enable =  Integer.valueOf(split[3]).intValue() == 1 ? true : false;
+                subSystems.add(new SubSystem(type, hierarchyID, cgroupNum, enable));
+            }
+            return subSystems;
+        } catch (Exception e) {
+            LOG.error("Get subSystems error {}", e);
+        }
+        return null;
+    }
+
+    @Override
+    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
+        Set<SubSystem> subSystems = this.getSubSystems();
+        for (SubSystem subSystem : subSystems) {
+            if (subSystem.getType() == subSystemType) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
+        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
+    }
+
+    @Override
+    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
+        List<Hierarchy> hierarchies = this.getHierarchies();
+        for (Hierarchy hierarchy : hierarchies) {
+            Hierarchy ret = hierarchy;
+            for (SubSystemType subSystem : subSystems) {
+                if (!hierarchy.getSubSystems().contains(subSystem)) {
+                    ret = null;
+                    break;
+                }
+            }
+            if (ret != null) {
+                return ret;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public boolean isMounted(Hierarchy hierarchy) {
+        if (Utils.checkDirExists(hierarchy.getDir())) {
+            List<Hierarchy> hierarchies = this.getHierarchies();
+            for (Hierarchy h : hierarchies) {
+                if (h.equals(hierarchy)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public void mount(Hierarchy hierarchy) throws IOException {
+        if (this.isMounted(hierarchy)) {
+            LOG.error("{} is already mounted", hierarchy.getDir());
+            return;
+        }
+        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
+        for (SubSystemType type : subSystems) {
+            Hierarchy hierarchyWithSubSystem = this.getHierarchyWithSubSystem(type);
+            if (hierarchyWithSubSystem != null) {
+                LOG.error("subSystem: {} is already mounted on hierarchy: {}", type.name(), hierarchyWithSubSystem);
+                subSystems.remove(type);
+            }
+        }
+        if (subSystems.size() == 0) {
+            return;
+        }
+        if (!Utils.checkDirExists(hierarchy.getDir())) {
+            new File(hierarchy.getDir()).mkdirs();
+        }
+        String subSystemsName = CgroupUtils.subSystemsToString(subSystems);
+        SystemOperation.mount(subSystemsName, hierarchy.getDir(), "cgroup", subSystemsName);
+
+    }
+
+    @Override
+    public void umount(Hierarchy hierarchy) throws IOException {
+        if (this.isMounted(hierarchy)) {
+            hierarchy.getRootCgroups().delete();
+            SystemOperation.umount(hierarchy.getDir());
+            CgroupUtils.deleteDir(hierarchy.getDir());
+        } else {
+            LOG.error("{} is not mounted", hierarchy.getDir());
+        }
+    }
+
+    @Override
+    public void createCgroup(CgroupCommon cgroup) throws SecurityException {
+        if (cgroup.isRoot()) {
+            LOG.error("You can't create rootCgroup in this function");
+            throw new RuntimeException("You can't create rootCgroup in this function");
+        }
+        CgroupCommon parent = cgroup.getParent();
+        while (parent != null) {
+            if (!Utils.checkDirExists(parent.getDir())) {
+                throw new RuntimeException("Parent " + parent.getDir() + "does not exist");
+            }
+            parent = parent.getParent();
+        }
+        Hierarchy h = cgroup.getHierarchy();
+        if (!isMounted(h)) {
+            throw new RuntimeException("hierarchy " + h.getDir() + " is not mounted");
+        }
+        if (Utils.checkDirExists(cgroup.getDir())) {
+            throw new RuntimeException("cgroup {} already exists " + cgroup.getDir());
+        }
+
+        if (!(new File(cgroup.getDir())).mkdir()) {
+            throw new RuntimeException("Could not create cgroup dir at " + cgroup.getDir());
+        }
+    }
+
+    @Override
+    public void deleteCgroup(CgroupCommon cgroup) throws IOException {
+        cgroup.delete();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java
new file mode 100755
index 0000000..c8bb304
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java
@@ -0,0 +1,270 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+import org.apache.storm.container.cgroup.core.CgroupCore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class CgroupCommon implements CgroupCommonOperation {
+
+    public static final String TASKS = "/tasks";
+    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
+    public static final String RELEASE_AGENT = "/release_agent";
+    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
+    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
+    public static final String CGROUP_PROCS = "/cgroup.procs";
+
+    private final Hierarchy hierarchy;
+
+    private final String name;
+
+    private final String dir;
+
+    private final CgroupCommon parent;
+
+    private final boolean isRoot;
+
+    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
+
+    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
+        this.name = parent.getName() + "/" + name;
+        this.hierarchy = hierarchy;
+        this.parent = parent;
+        this.dir = parent.getDir() + "/" + name;
+        this.isRoot = false;
+    }
+
+    /**
+     * rootCgroup
+     */
+    public CgroupCommon(Hierarchy hierarchy, String dir) {
+        this.name = "";
+        this.hierarchy = hierarchy;
+        this.parent = null;
+        this.dir = dir;
+        this.isRoot = true;
+    }
+
+    @Override
+    public void addTask(int taskId) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
+    }
+
+    @Override
+    public Set<Integer> getTasks() throws IOException {
+        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
+        Set<Integer> tasks = new HashSet<Integer>();
+        for (String task : stringTasks) {
+            tasks.add(Integer.valueOf(task));
+        }
+        return tasks;
+    }
+
+    @Override
+    public void addProcs(int pid) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
+    }
+
+    @Override
+    public Set<Long> getPids() throws IOException {
+        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
+        Set<Long> pids = new HashSet<>();
+        for (String task : stringPids) {
+            pids.add(Long.valueOf(task));
+        }
+        return pids;
+    }
+
+    @Override
+    public void setNotifyOnRelease(boolean flag) throws IOException {
+
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
+    }
+
+    @Override
+    public boolean getNotifyOnRelease() throws IOException {
+        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
+    }
+
+    @Override
+    public void setReleaseAgent(String command) throws IOException {
+        if (!this.isRoot) {
+            LOG.warn("Cannot set {} in {} since its not the root group", RELEASE_AGENT, this.isRoot);
+            return;
+        }
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
+    }
+
+    @Override
+    public String getReleaseAgent() throws IOException {
+        if (!this.isRoot) {
+            LOG.warn("Cannot get {} in {} since its not the root group", RELEASE_AGENT, this.isRoot);
+            return null;
+        }
+        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
+    }
+
+    @Override
+    public void setCgroupCloneChildren(boolean flag) throws IOException {
+        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
+            return;
+        }
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
+    }
+
+    @Override
+    public boolean getCgroupCloneChildren() throws IOException {
+        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
+    }
+
+    @Override
+    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        sb.append(eventFd);
+        sb.append(' ');
+        sb.append(controlFd);
+        for (String arg : args) {
+            sb.append(' ');
+            sb.append(arg);
+        }
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
+    }
+
+    public Hierarchy getHierarchy() {
+        return hierarchy;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getDir() {
+        return dir;
+    }
+
+    public CgroupCommon getParent() {
+        return parent;
+    }
+
+    public Set<CgroupCommon> getChildren() {
+
+        File file = new File(this.dir);
+        File[] files = file.listFiles();
+        if (files == null) {
+            LOG.info("{} is not a directory", this.dir);
+            return null;
+        }
+        Set<CgroupCommon> children = new HashSet<CgroupCommon>();
+        for (File child : files) {
+            if (child.isDirectory()) {
+                children.add(new CgroupCommon(child.getName(), this.hierarchy, this));
+            }
+        }
+        return children;
+    }
+
+    public boolean isRoot() {
+        return isRoot;
+    }
+
+    public Map<SubSystemType, CgroupCore> getCores() {
+        return CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
+    }
+
+    public void delete() throws IOException {
+        this.free();
+        if (!this.isRoot) {
+            this.parent.getChildren().remove(this);
+        }
+    }
+
+    private void free() throws IOException {
+        for (CgroupCommon child : getChildren()) {
+            child.free();
+        }
+        if (this.isRoot) {
+            return;
+        }
+        Set<Integer> tasks = this.getTasks();
+        if (tasks != null) {
+            for (Integer task : tasks) {
+                this.parent.addTask(task);
+            }
+        }
+        CgroupUtils.deleteDir(this.dir);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        boolean ret = false;
+        if (o != null && (o instanceof CgroupCommon)) {
+
+            boolean hierarchyFlag =false;
+            if (((CgroupCommon)o).hierarchy != null && this.hierarchy != null) {
+                hierarchyFlag = ((CgroupCommon)o).hierarchy.equals(this.hierarchy);
+            } else if (((CgroupCommon)o).hierarchy == null && this.hierarchy == null) {
+                hierarchyFlag = true;
+            } else {
+                hierarchyFlag = false;
+            }
+
+            boolean nameFlag = false;
+            if (((CgroupCommon)o).name != null && this.name != null) {
+                nameFlag = ((CgroupCommon)o).name.equals(this.name);
+            } else if (((CgroupCommon)o).name == null && this.name == null) {
+                nameFlag = true;
+            } else {
+                nameFlag = false;
+            }
+
+            boolean dirFlag = false;
+            if (((CgroupCommon)o).dir != null && this.dir != null) {
+                dirFlag = ((CgroupCommon)o).dir.equals(this.dir);
+            } else if (((CgroupCommon)o).dir == null && this.dir == null) {
+                dirFlag = true;
+            } else {
+                dirFlag = false;
+            }
+            ret = hierarchyFlag && nameFlag && dirFlag;
+        }
+        return ret;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (this.name != null ? this.name.hashCode() : 0);
+        result = prime * result + (this.hierarchy != null ? this.hierarchy.hashCode() : 0);
+        result = prime * result + (this.dir != null ? this.dir.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return this.getName();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java
new file mode 100755
index 0000000..eecba69
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+import java.io.IOException;
+import java.util.Set;
+
+public interface CgroupCommonOperation {
+
+    /**
+     * add task to cgroup
+     * @param taskid task id of task to add
+     */
+    public void addTask(int taskid) throws IOException;
+
+    /**
+     * Get a list of task ids running in CGroup
+     */
+    public Set<Integer> getTasks() throws IOException;
+
+    /**
+     * add a process to cgroup
+     * @param pid the PID of the process to add
+     */
+    public void addProcs(int pid) throws IOException;
+
+    /**
+     * get the PIDs of processes running in cgroup
+     */
+    public Set<Long> getPids() throws IOException;
+
+    /**
+     * to set notify_on_release config in cgroup
+     */
+    public void setNotifyOnRelease(boolean flag) throws IOException;
+
+    /**
+     * to get the notify_on_release config
+     */
+    public boolean getNotifyOnRelease() throws IOException;
+
+    /**
+     * set a command for the release agent to execute
+     */
+    public void setReleaseAgent(String command) throws IOException;
+
+    /**
+     * get the command for the relase agent to execute
+     */
+    public String getReleaseAgent() throws IOException;
+
+    /**
+     * Set the cgroup.clone_children config
+     */
+    public void setCgroupCloneChildren(boolean flag) throws IOException;
+
+    /**
+     * get the cgroup.clone_children config
+     */
+    public boolean getCgroupCloneChildren() throws IOException;
+
+    /**
+     * set event control config
+     */
+    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCoreFactory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCoreFactory.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCoreFactory.java
new file mode 100755
index 0000000..4d35c77
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCoreFactory.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+import org.apache.storm.container.cgroup.core.BlkioCore;
+import org.apache.storm.container.cgroup.core.CgroupCore;
+import org.apache.storm.container.cgroup.core.CpuCore;
+import org.apache.storm.container.cgroup.core.CpuacctCore;
+import org.apache.storm.container.cgroup.core.CpusetCore;
+import org.apache.storm.container.cgroup.core.DevicesCore;
+import org.apache.storm.container.cgroup.core.FreezerCore;
+import org.apache.storm.container.cgroup.core.MemoryCore;
+import org.apache.storm.container.cgroup.core.NetClsCore;
+import org.apache.storm.container.cgroup.core.NetPrioCore;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class CgroupCoreFactory {
+
+    public static CgroupCore getInstance(SubSystemType type, String dir) {
+        switch (type) {
+        case blkio:
+            return new BlkioCore(dir);
+        case cpuacct:
+            return new CpuacctCore(dir);
+        case cpuset:
+            return new CpusetCore(dir);
+        case cpu:
+            return new CpuCore(dir);
+        case devices:
+            return new DevicesCore(dir);
+        case freezer:
+            return new FreezerCore(dir);
+        case memory:
+            return new MemoryCore(dir);
+        case net_cls:
+            return new NetClsCore(dir);
+        case net_prio:
+            return new NetPrioCore(dir);
+        default:
+           return null;
+        }
+    }
+
+    public static Map<SubSystemType, CgroupCore> getInstance(Set<SubSystemType> types, String dir) {
+        Map<SubSystemType, CgroupCore> result = new HashMap<SubSystemType, CgroupCore>();
+        for (SubSystemType type : types) {
+            CgroupCore inst = getInstance(type, dir);
+            if (inst != null) {
+                result.put(type, inst);
+            }
+        }
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java
new file mode 100755
index 0000000..00ac9fd
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * An interface to implement the basic functions to manage cgroups such as mount and mounting a hiearchy
+ * and creating cgroups.  Also contains functions to access basic information of cgroups.
+ */
+public interface CgroupOperation {
+
+    /**
+     * Get a list of hierarchies
+     */
+    public List<Hierarchy> getHierarchies();
+
+    /**
+     * get a list of available subsystems
+     */
+    public Set<SubSystem> getSubSystems();
+
+    /**
+     * Check if a subsystem is enabled
+     */
+    public boolean isSubSystemEnabled(SubSystemType subsystem);
+
+    /**
+     * get the first hierarchy that has a certain subsystem isMounted
+     */
+    public Hierarchy getHierarchyWithSubSystem(SubSystemType subsystem);
+
+    /**
+     * get the first hierarchy that has a certain list of subsystems isMounted
+     */
+    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems);
+
+    /**
+     * check if a hiearchy is mounted
+     */
+    public boolean isMounted(Hierarchy hierarchy);
+
+    /**
+     * mount a hierarchy
+     */
+    public void mount(Hierarchy hierarchy) throws IOException;
+
+    /**
+     * umount a heirarchy
+     */
+    public void umount(Hierarchy hierarchy) throws IOException;
+
+    /**
+     * create a cgroup
+     */
+    public void createCgroup(CgroupCommon cgroup) throws SecurityException;
+
+    /**
+     * delete a cgroup
+     */
+    public void deleteCgroup(CgroupCommon cgroup) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java
new file mode 100644
index 0000000..5a4744c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+import com.google.common.io.Files;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class CgroupUtils {
+
+    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
+    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
+
+    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
+
+    public static void deleteDir(String dir) {
+        File d = new File(dir);
+        if (!d.exists()) {
+            LOG.warn("dir {} does not exist!", dir);
+            return;
+        }
+        if (!d.isDirectory()) {
+            throw new RuntimeException("dir " + dir + " is not a directory!");
+        }
+        if (!d.delete()) {
+            throw new RuntimeException("Cannot delete dir " + dir);
+        }
+    }
+
+    /**
+     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
+     */
+    public static Set<SubSystemType> getSubSystemsFromString(String str) {
+        Set<SubSystemType> result = new HashSet<SubSystemType>();
+        String[] subSystems = str.split(",");
+        for (String subSystem : subSystems) {
+            //return null to mount options in string that is not part of cgroups
+            SubSystemType type = SubSystemType.getSubSystem(subSystem);
+            if (type != null) {
+                result.add(type);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Get a string that is a comma delimited list of subsystems
+     */
+    public static String subSystemsToString(Set<SubSystemType> subSystems) {
+        StringBuilder sb = new StringBuilder();
+        if (subSystems.size() == 0) {
+            return sb.toString();
+        }
+        for (SubSystemType type : subSystems) {
+            sb.append(type.name()).append(",");
+        }
+        return sb.toString().substring(0, sb.length() - 1);
+    }
+
+    public static boolean enabled() {
+        return Utils.checkFileExists(CGROUP_STATUS_FILE);
+    }
+
+    public static List<String> readFileByLine(String filePath) throws IOException {
+        return Files.readLines(new File(filePath), Charset.defaultCharset());
+    }
+
+    public static void writeFileByLine(String filePath, List<String> linesToWrite) throws IOException {
+        LOG.debug("For CGroups - writing {} to {} ", linesToWrite, filePath);
+        File file = new File(filePath);
+        if (!file.exists()) {
+            LOG.error("{} does not exist", filePath);
+            return;
+        }
+        try (FileWriter writer = new FileWriter(file, true);
+             BufferedWriter bw = new BufferedWriter(writer)) {
+            for (String string : linesToWrite) {
+                bw.write(string);
+                bw.newLine();
+                bw.flush();
+            }
+        }
+    }
+
+    public static void writeFileByLine(String filePath, String lineToWrite) throws IOException {
+        writeFileByLine(filePath, Arrays.asList(lineToWrite));
+    }
+
+    public static String getDir(String dir, String constant) {
+        return dir + constant;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java
new file mode 100755
index 0000000..57eb8ff
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+/**
+ * a class that represents a device in linux
+ */
+public class Device {
+
+    public final int major;
+    public final int minor;
+
+    public Device(int major, int minor) {
+        this.major = major;
+        this.minor = minor;
+    }
+
+    public Device(String str) {
+        String[] strArgs = str.split(":");
+        this.major = Integer.valueOf(strArgs[0]);
+        this.minor = Integer.valueOf(strArgs[1]);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append(major).append(":").append(minor);
+        return sb.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + major;
+        result = prime * result + minor;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        Device other = (Device) obj;
+        if (major != other.major) {
+            return false;
+        }
+        if (minor != other.minor) {
+            return false;
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java
new file mode 100755
index 0000000..b2b245c
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+import java.util.Set;
+
+/**
+ * A class that describes a cgroup hierarchy
+ */
+public class Hierarchy {
+
+    private final String name;
+
+    private final Set<SubSystemType> subSystems;
+
+    private final String type;
+
+    private final String dir;
+
+    private final CgroupCommon rootCgroups;
+
+    public Hierarchy(String name, Set<SubSystemType> subSystems, String dir) {
+        this.name = name;
+        this.subSystems = subSystems;
+        this.dir = dir;
+        this.rootCgroups = new CgroupCommon(this, dir);
+        this.type = CgroupUtils.subSystemsToString(subSystems);
+    }
+
+    /**
+     * get subsystems
+     */
+    public Set<SubSystemType> getSubSystems() {
+        return subSystems;
+    }
+
+    /**
+     * get all subsystems in hierarchy as a comma delimited list
+     */
+    public String getType() {
+        return type;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((dir == null) ? 0 : dir.hashCode());
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + ((type == null) ? 0 : type.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        Hierarchy other = (Hierarchy) obj;
+        if (dir == null) {
+            if (other.dir != null) {
+                return false;
+            }
+        } else if (!dir.equals(other.dir)) {
+            return false;
+        }
+        if (name == null) {
+            if (other.name != null) {
+                return false;
+            }
+        } else if (!name.equals(other.name)) {
+            return false;
+        }
+        if (type == null) {
+            if (other.type != null) {
+                return false;
+            }
+        } else if (!type.equals(other.type)) {
+            return false;
+        }
+        return true;
+    }
+
+    public String getDir() {
+        return dir;
+    }
+
+    public CgroupCommon getRootCgroups() {
+        return rootCgroups;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public boolean isSubSystemMounted(SubSystemType subsystem) {
+        for (SubSystemType type : this.subSystems) {
+            if (type == subsystem) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return this.dir;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java
new file mode 100755
index 0000000..e354fb0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+/**
+ * a class that implements operations that can be performed on a cgroup subsystem
+ */
+public class SubSystem {
+
+    private SubSystemType type;
+
+    private int hierarchyID;
+
+    private int cgroupsNum;
+
+    private boolean enable;
+
+    public SubSystem(SubSystemType type, int hierarchyID, int cgroupNum, boolean enable) {
+        this.type = type;
+        this.hierarchyID = hierarchyID;
+        this.cgroupsNum = cgroupNum;
+        this.enable = enable;
+    }
+
+    public SubSystemType getType() {
+        return type;
+    }
+
+    public void setType(SubSystemType type) {
+        this.type = type;
+    }
+
+    public int getHierarchyID() {
+        return hierarchyID;
+    }
+
+    public void setHierarchyID(int hierarchyID) {
+        this.hierarchyID = hierarchyID;
+    }
+
+    public int getCgroupsNum() {
+        return cgroupsNum;
+    }
+
+    public void setCgroupsNum(int cgroupsNum) {
+        this.cgroupsNum = cgroupsNum;
+    }
+
+    public boolean isEnable() {
+        return enable;
+    }
+
+    public void setEnable(boolean enable) {
+        this.enable = enable;
+    }
+
+    @Override
+    public boolean equals(Object object) {
+        boolean ret = false;
+        if (object != null && object instanceof SubSystem) {
+            ret = ((this.type == ((SubSystem)object).getType())
+                    && (this.hierarchyID == ((SubSystem)object).getHierarchyID()));
+        }
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java
new file mode 100755
index 0000000..914abcc
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup;
+
+/**
+ * A enum class to described the subsystems that can be used
+ */
+public enum SubSystemType {
+
+    // net_cls,ns is not supported in ubuntu
+    blkio, cpu, cpuacct, cpuset, devices, freezer, memory, perf_event, net_cls, net_prio;
+
+
+    public static SubSystemType getSubSystem(String str) {
+        try {
+            return SubSystemType.valueOf(str);
+        } catch (Exception e) {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java
new file mode 100644
index 0000000..8ebd763
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.container.cgroup;
+
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.channels.ClosedByInterruptException;
+
+/**
+ * A class that implements system operations for using cgroups
+ */
+public class SystemOperation {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
+
+    public static boolean isRoot() throws IOException {
+        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
+        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
+    }
+
+    public static void mount(String name, String target, String type, String options) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        sb.append("mount -t ")
+                .append(type)
+                .append(" -o ")
+                .append(options)
+                .append(" ")
+                .append(name)
+                .append(" ")
+                .append(target);
+        SystemOperation.exec(sb.toString());
+    }
+
+    public static void umount(String pathToDir) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        sb.append("umount ").append(pathToDir);
+        SystemOperation.exec(sb.toString());
+    }
+
+    public static String exec(String cmd) throws IOException {
+        LOG.debug("Shell cmd: {}", cmd);
+        Process process = new ProcessBuilder(new String[] { "/bin/bash", "-c", cmd }).start();
+        try {
+            process.waitFor();
+            String output = IOUtils.toString(process.getInputStream());
+            String errorOutput = IOUtils.toString(process.getErrorStream());
+            LOG.debug("Shell Output: {}", output);
+            if (errorOutput.length() != 0) {
+                LOG.error("Shell Error Output: {}", errorOutput);
+                throw new IOException(errorOutput);
+            }
+            return output;
+        } catch (InterruptedException | ClosedByInterruptException ie) {
+            throw new IOException(ie);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java
new file mode 100755
index 0000000..c426610
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java
@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+import org.apache.storm.container.cgroup.Device;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BlkioCore implements CgroupCore {
+
+    public static final String BLKIO_WEIGHT = "/blkio.weight";
+    public static final String BLKIO_WEIGHT_DEVICE = "/blkio.weight_device";
+    public static final String BLKIO_RESET_STATS = "/blkio.reset_stats";
+
+    public static final String BLKIO_THROTTLE_READ_BPS_DEVICE = "/blkio.throttle.read_bps_device";
+    public static final String BLKIO_THROTTLE_WRITE_BPS_DEVICE = "/blkio.throttle.write_bps_device";
+    public static final String BLKIO_THROTTLE_READ_IOPS_DEVICE = "/blkio.throttle.read_iops_device";
+    public static final String BLKIO_THROTTLE_WRITE_IOPS_DEVICE = "/blkio.throttle.write_iops_device";
+
+    public static final String BLKIO_THROTTLE_IO_SERVICED = "/blkio.throttle.io_serviced";
+    public static final String BLKIO_THROTTLE_IO_SERVICE_BYTES = "/blkio.throttle.io_service_bytes";
+
+    public static final String BLKIO_TIME = "/blkio.time";
+    public static final String BLKIO_SECTORS = "/blkio.sectors";
+    public static final String BLKIO_IO_SERVICED = "/blkio.io_serviced";
+    public static final String BLKIO_IO_SERVICE_BYTES = "/blkio.io_service_bytes";
+    public static final String BLKIO_IO_SERVICE_TIME = "/blkio.io_service_time";
+    public static final String BLKIO_IO_WAIT_TIME = "/blkio.io_wait_time";
+    public static final String BLKIO_IO_MERGED = "/blkio.io_merged";
+    public static final String BLKIO_IO_QUEUED = "/blkio.io_queued";
+
+    private final String dir;
+
+    public BlkioCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.blkio;
+    }
+
+    /* weight: 100-1000 */
+    public void setBlkioWeight(int weight) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT), String.valueOf(weight));
+    }
+
+    public int getBlkioWeight() throws IOException {
+        return Integer.valueOf(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT)).get(0)).intValue();
+    }
+
+    public void setBlkioWeightDevice(Device device, int weight) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT_DEVICE), makeContext(device, weight));
+    }
+
+    public Map<Device, Integer> getBlkioWeightDevice() throws IOException {
+        List<String> strings = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT_DEVICE));
+        Map<Device, Integer> result = new HashMap<Device, Integer>();
+        for (String string : strings) {
+            String[] strArgs = string.split(" ");
+            Device device = new Device(strArgs[0]);
+            Integer weight = Integer.valueOf(strArgs[1]);
+            result.put(device, weight);
+        }
+        return result;
+    }
+
+    public void setReadBps(Device device, long bps) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE), makeContext(device, bps));
+    }
+
+    public Map<Device, Long> getReadBps() throws IOException {
+        return parseConfig(BLKIO_THROTTLE_READ_BPS_DEVICE);
+    }
+
+    public void setWriteBps(Device device, long bps) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE), makeContext(device, bps));
+    }
+
+    public Map<Device, Long> getWriteBps() throws IOException {
+        return parseConfig(BLKIO_THROTTLE_WRITE_BPS_DEVICE);
+    }
+
+    public void setReadIOps(Device device, long iops) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE), makeContext(device, iops));
+    }
+
+    public Map<Device, Long> getReadIOps() throws IOException {
+        return parseConfig(BLKIO_THROTTLE_READ_IOPS_DEVICE);
+    }
+
+    public void setWriteIOps(Device device, long iops) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE), makeContext(device, iops));
+    }
+
+    public Map<Device, Long> getWriteIOps() throws IOException {
+        return parseConfig(BLKIO_THROTTLE_WRITE_IOPS_DEVICE);
+    }
+
+    public Map<Device, Map<RecordType, Long>> getThrottleIOServiced() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICED)));
+    }
+
+    public Map<Device, Map<RecordType, Long>> getThrottleIOServiceByte() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICE_BYTES)));
+    }
+
+    public Map<Device, Long> getBlkioTime() throws IOException {
+        return parseConfig(BLKIO_TIME);
+    }
+
+    public Map<Device, Long> getBlkioSectors() throws IOException {
+        return parseConfig(BLKIO_SECTORS);
+    }
+
+    public Map<Device, Map<RecordType, Long>> getIOServiced() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICED)));
+    }
+
+    public Map<Device, Map<RecordType, Long>> getIOServiceBytes() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_BYTES)));
+    }
+
+    public Map<Device, Map<RecordType, Long>> getIOServiceTime() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_TIME)));
+    }
+
+    public Map<Device, Map<RecordType, Long>> getIOWaitTime() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_WAIT_TIME)));
+    }
+
+    public Map<Device, Map<RecordType, Long>> getIOMerged() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_MERGED)));
+    }
+
+    public Map<Device, Map<RecordType, Long>> getIOQueued() throws IOException {
+        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_QUEUED)));
+    }
+
+    public void resetStats() throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_RESET_STATS), "1");
+    }
+
+    private String makeContext(Device device, Object data) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(device.toString()).append(" ").append(data);
+        return sb.toString();
+    }
+
+    private Map<Device, Long> parseConfig(String config) throws IOException {
+        List<String> strings = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, config));
+        Map<Device, Long> result = new HashMap<Device, Long>();
+        for (String string : strings) {
+            String[] strArgs = string.split(" ");
+            Device device = new Device(strArgs[0]);
+            Long value = Long.valueOf(strArgs[1]);
+            result.put(device, value);
+        }
+        return result;
+    }
+
+    private Map<Device, Map<RecordType, Long>> analyseRecord(List<String> strs) {
+        Map<Device, Map<RecordType, Long>> result = new HashMap<Device, Map<RecordType, Long>>();
+        for (String str : strs) {
+            String[] strArgs = str.split(" ");
+            if (strArgs.length != 3) {
+                continue;
+            }
+            Device device = new Device(strArgs[0]);
+            RecordType key = RecordType.getType(strArgs[1]);
+            Long value = Long.parseLong(strArgs[2]);
+            Map<RecordType, Long> record = result.get(device);
+            if (record == null) {
+                record = new HashMap<RecordType, Long>();
+                result.put(device, record);
+            }
+            record.put(key, value);
+        }
+        return result;
+    }
+
+    public enum RecordType {
+        read, write, sync, async, total;
+
+        public static RecordType getType(String type) {
+            try {
+                return RecordType.valueOf(type.toLowerCase());
+            } catch (Exception e) {
+                return null;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CgroupCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CgroupCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CgroupCore.java
new file mode 100755
index 0000000..a6b098e
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CgroupCore.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.SubSystemType;
+
+public interface CgroupCore {
+
+    public SubSystemType getType();
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuCore.java
new file mode 100755
index 0000000..1d21251
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuCore.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+
+import java.io.IOException;
+import java.util.List;
+
+public class CpuCore implements CgroupCore {
+
+    public static final String CPU_SHARES = "/cpu.shares";
+    public static final String CPU_RT_RUNTIME_US = "/cpu.rt_runtime_us";
+    public static final String CPU_RT_PERIOD_US = "/cpu.rt_period_us";
+    public static final String CPU_CFS_PERIOD_US = "/cpu.cfs_period_us";
+    public static final String CPU_CFS_QUOTA_US = "/cpu.cfs_quota_us";
+    public static final String CPU_STAT = "/cpu.stat";
+
+    private final String dir;
+
+    public CpuCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.cpu;
+    }
+
+    public void setCpuShares(int weight) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPU_SHARES), String.valueOf(weight));
+    }
+
+    public int getCpuShares() throws IOException {
+        return Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPU_SHARES)).get(0));
+    }
+
+    public void setCpuRtRuntimeUs(long us) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPU_RT_RUNTIME_US), String.valueOf(us));
+    }
+
+    public long getCpuRtRuntimeUs() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPU_RT_RUNTIME_US)).get(0));
+    }
+
+    public void setCpuRtPeriodUs(long us) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPU_RT_PERIOD_US), String.valueOf(us));
+    }
+
+    public Long getCpuRtPeriodUs() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPU_RT_PERIOD_US)).get(0));
+    }
+
+    public void setCpuCfsPeriodUs(long us) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPU_CFS_PERIOD_US), String.valueOf(us));
+    }
+
+    public Long getCpuCfsPeriodUs() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPU_CFS_PERIOD_US)).get(0));
+    }
+
+    public void setCpuCfsQuotaUs(long us) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPU_CFS_QUOTA_US), String.valueOf(us));
+    }
+
+    public Long getCpuCfsQuotaUs() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPU_CFS_QUOTA_US)).get(0));
+    }
+
+    public Stat getCpuStat() throws IOException {
+        return new Stat(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPU_STAT)));
+    }
+
+    public static class Stat {
+        public final int nrPeriods;
+        public final int nrThrottled;
+        public final int throttledTime;
+
+        public Stat(List<String> statStr) {
+            this.nrPeriods = Integer.parseInt(statStr.get(0).split(" ")[1]);
+            this.nrThrottled = Integer.parseInt(statStr.get(1).split(" ")[1]);
+            this.throttledTime = Integer.parseInt(statStr.get(2).split(" ")[1]);
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + nrPeriods;
+            result = prime * result + nrThrottled;
+            result = prime * result + throttledTime;
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) {
+                return true;
+            }
+            if (obj == null) {
+                return false;
+            }
+            if (getClass() != obj.getClass()) {
+                return false;
+            }
+            Stat other = (Stat) obj;
+            if (nrPeriods != other.nrPeriods) {
+                return false;
+            }
+            if (nrThrottled != other.nrThrottled) {
+                return false;
+            }
+            if (throttledTime != other.throttledTime) {
+                return false;
+            }
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuacctCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuacctCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuacctCore.java
new file mode 100755
index 0000000..2e683f4
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpuacctCore.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CpuacctCore implements CgroupCore {
+
+    public static final String CPUACCT_USAGE = "/cpuacct.usage";
+    public static final String CPUACCT_STAT = "/cpuacct.stat";
+    public static final String CPUACCT_USAGE_PERCPU = "/cpuacct.usage_percpu";
+
+    private final String dir;
+
+    public CpuacctCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.cpuacct;
+    }
+
+    public Long getCpuUsage() throws IOException {
+        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_USAGE)).get(0));
+    }
+
+    public Map<StatType, Long> getCpuStat() throws IOException {
+        List<String> strs = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_STAT));
+        Map<StatType, Long> result = new HashMap<StatType, Long>();
+        result.put(StatType.user, Long.parseLong(strs.get(0).split(" ")[1]));
+        result.put(StatType.system, Long.parseLong(strs.get(1).split(" ")[1]));
+        return result;
+    }
+
+    public Long[] getPerCpuUsage() throws IOException {
+        String str = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_USAGE_PERCPU)).get(0);
+        String[] strArgs = str.split(" ");
+        Long[] result = new Long[strArgs.length];
+        for (int i = 0; i < result.length; i++) {
+            result[i] = Long.parseLong(strArgs[i]);
+        }
+        return result;
+    }
+
+    public static enum StatType {
+        user, system;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4de339a8/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java
new file mode 100755
index 0000000..d089e95
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.container.cgroup.core;
+
+import org.apache.storm.container.cgroup.CgroupUtils;
+import org.apache.storm.container.cgroup.SubSystemType;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+public class CpusetCore implements CgroupCore {
+
+    public static final String CPUSET_CPUS = "/cpuset.cpus";
+    public static final String CPUSET_MEMS = "/cpuset.mems";
+    public static final String CPUSET_MEMORY_MIGRATE = "/cpuset.memory_migrate";
+    public static final String CPUSET_CPU_EXCLUSIVE = "/cpuset.cpu_exclusive";
+    public static final String CPUSET_MEM_EXCLUSIVE = "/cpuset.mem_exclusive";
+    public static final String CPUSET_MEM_HARDWALL = "/cpuset.mem_hardwall";
+    public static final String CPUSET_MEMORY_PRESSURE = "/cpuset.memory_pressure";
+    public static final String CPUSET_MEMORY_PRESSURE_ENABLED = "/cpuset.memory_pressure_enabled";
+    public static final String CPUSET_MEMORY_SPREAD_PAGE = "/cpuset.memory_spread_page";
+    public static final String CPUSET_MEMORY_SPREAD_SLAB = "/cpuset.memory_spread_slab";
+    public static final String CPUSET_SCHED_LOAD_BALANCE = "/cpuset.sched_load_balance";
+    public static final String CPUSET_SCHED_RELAX_DOMAIN_LEVEL = "/cpuset.sched_relax_domain_level";
+
+    private final String dir;
+
+    public CpusetCore(String dir) {
+        this.dir = dir;
+    }
+
+    @Override
+    public SubSystemType getType() {
+        return SubSystemType.cpuset;
+    }
+
+    public void setCpus(int[] nums) throws IOException {
+        setConfigs(nums, CPUSET_CPUS);
+    }
+
+    public int[] getCpus() throws IOException {
+        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPUS)).get(0);
+        return parseNums(output);
+    }
+
+    public void setMems(int[] nums) throws IOException {
+        setConfigs(nums, CPUSET_MEMS);
+    }
+
+    private void setConfigs(int[] nums, String config) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        for (int num : nums) {
+            sb.append(num);
+            sb.append(',');
+        }
+        sb.deleteCharAt(sb.length() - 1);
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, config), sb.toString());
+    }
+
+    public int[] getMems() throws IOException {
+        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMS)).get(0);
+        return parseNums(output);
+    }
+
+    public void setMemMigrate(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isMemMigrate() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE)).get(0));
+        return output > 0;
+    }
+
+    public void setCpuExclusive(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isCpuExclusive() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE)).get(0));
+        return output > 0;
+    }
+
+    public void setMemExclusive(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isMemExclusive() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE)).get(0));
+        return output > 0;
+    }
+
+    public void setMemHardwall(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isMemHardwall() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL)).get(0));
+        return output > 0;
+    }
+
+    public int getMemPressure() throws IOException {
+        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE)).get(0);
+        return Integer.parseInt(output);
+    }
+
+    public void setMemPressureEnabled(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isMemPressureEnabled() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED)).get(0));
+        return output > 0;
+    }
+
+    public void setMemSpreadPage(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isMemSpreadPage() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE)).get(0));
+        return output > 0;
+    }
+
+    public void setMemSpreadSlab(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isMemSpreadSlab() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB)).get(0));
+        return output > 0;
+    }
+
+    public void setSchedLoadBlance(boolean flag) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE), String.valueOf(flag ? 1 : 0));
+    }
+
+    public boolean isSchedLoadBlance() throws IOException {
+        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE)).get(0));
+        return output > 0;
+    }
+
+    public void setSchedRelaxDomainLevel(int value) throws IOException {
+        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL), String.valueOf(value));
+    }
+
+    public int getSchedRelaxDomainLevel() throws IOException {
+        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL)).get(0);
+        return Integer.parseInt(output);
+    }
+
+    public static int[] parseNums(String outputStr) {
+        char[] output = outputStr.toCharArray();
+        LinkedList<Integer> numList = new LinkedList<Integer>();
+        int value = 0;
+        int start = 0;
+        boolean isHyphen = false;
+        for (char ch : output) {
+            if (ch == ',') {
+                if (isHyphen) {
+                    for (; start <= value; start++) {
+                        numList.add(start);
+                    }
+                    isHyphen = false;
+                } else {
+                    numList.add(value);
+                }
+                value = 0;
+            } else if (ch == '-') {
+                isHyphen = true;
+                start = value;
+                value = 0;
+            } else {
+                value = value * 10 + (ch - '0');
+            }
+        }
+        if (output[output.length - 1] != ',') {
+            if (isHyphen) {
+                for (; start <= value; start++) {
+                    numList.add(start);
+                }
+            } else {
+                numList.add(value);
+            }
+        }
+
+        int[] nums = new int[numList.size()];
+        int index = 0;
+        for (int num : numList) {
+            nums[index] = num;
+            index++;
+        }
+        return nums;
+    }
+}