You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by kn...@apache.org on 2015/10/19 22:06:58 UTC

[01/18] storm git commit: Fix one test issue

Repository: storm
Updated Branches:
  refs/heads/master 1d393eee2 -> d23bd3ed2


Fix one test issue


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

Branch: refs/heads/master
Commit: f9e095dbb938134dd5442fe3a425f687dd17176b
Parents: 584ded8
Author: zhuol <zh...@yahoo-inc.com>
Authored: Fri Oct 9 11:09:56 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:38:44 2015 -0500

----------------------------------------------------------------------
 storm-core/test/clj/backtype/storm/supervisor_test.clj | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f9e095db/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index 8303754..907274e 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -668,23 +668,29 @@
      (bind changed (capture-changed-workers
                     (submit-mocked-assignment
                      (:nimbus cluster)
+                     (:storm-cluster-state cluster)
                      "topology1"
                      {TOPOLOGY-WORKERS 2}
                      topology1
                      {1 "1"
                       2 "1"}
                      {[1 1] ["sup1" 1]
-                      [2 2] ["sup1" 2]
+                      [2 2] ["sup1" 2]}
+                     {["sup1" 1] [0.0 0.0 0.0]
+                      ["sup1" 2] [0.0 0.0 0.0]
                       })
                     (submit-mocked-assignment
                      (:nimbus cluster)
+                     (:storm-cluster-state cluster)
                      "topology2"
                      {TOPOLOGY-WORKERS 2}
                      topology2
                      {1 "1"
                       2 "1"}
                      {[1 1] ["sup1" 1]
-                      [2 2] ["sup1" 2]
+                      [2 2] ["sup1" 2]}
+                     {["sup1" 1] [0.0 0.0 0.0]
+                      ["sup1" 2] [0.0 0.0 0.0]
                       })
                     (advance-cluster-time cluster 10)
                     ))


[15/18] storm git commit: Calling Slot Resource Allocation from RAS

Posted by kn...@apache.org.
Calling Slot Resource Allocation from RAS


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

Branch: refs/heads/master
Commit: e50bed80bea67ceef43cdca354c24c5eee4dbef4
Parents: f9e095d
Author: zhuol <zh...@yahoo-inc.com>
Authored: Fri Oct 9 13:35:54 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:44:12 2015 -0500

----------------------------------------------------------------------
 .../clj/backtype/storm/daemon/supervisor.clj    |  2 +-
 storm-core/src/jvm/backtype/storm/Config.java   |  2 +-
 .../storm/scheduler/resource/RAS_Node.java      | 28 ++++++++++++++++++++
 3 files changed, 30 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e50bed80/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 8a844f2..0461cd7 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -66,7 +66,7 @@
 
 (defn- read-my-executors [assignments-snapshot storm-id assignment-id]
   (let [assignment (get assignments-snapshot storm-id)
-        my-slots-resources (into {} 
+        my-slots-resources (into {}
                                  (filter (fn [[[node _] _]] (= node assignment-id))
                                          (:worker->resources assignment)))
         my-executors (filter (fn [[_ [node _]]] (= node assignment-id))

http://git-wip-us.apache.org/repos/asf/storm/blob/e50bed80/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 59dcc53..a1c732d 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1032,7 +1032,7 @@ public class Config extends HashMap<String, Object> {
     public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
 
     /**
-     * The jvm opts provided to workers launched by this supervisor. 
+     * 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),

http://git-wip-us.apache.org/repos/asf/storm/blob/e50bed80/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index 3c86528..1f2e795 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -230,6 +230,33 @@ public class RAS_Node {
         _topIdToUsedSlots.remove(topId);
     }
 
+    /**
+     * Allocate Mem and CPU resources to the assigned slot for the topology's executors.
+     * @param td the TopologyDetails that the slot is assigned to.
+     * @param executors the executors to run in that slot.
+     * @param slot the slot to allocate resource to
+     */
+    public void allocateResourceToSlot (TopologyDetails td, Collection<ExecutorDetails> executors, WorkerSlot slot) {
+        double onHeapMem = 0.0;
+        double offHeapMem = 0.0;
+        double cpu = 0.0;
+        for (ExecutorDetails exec : executors) {
+            Double onHeapMemForExec = td.getOnHeapMemoryRequirement(exec);
+            if (onHeapMemForExec != null) {
+                onHeapMem += onHeapMemForExec;
+            }
+            Double offHeapMemForExec = td.getOffHeapMemoryRequirement(exec);
+            if (offHeapMemForExec != null) {
+                offHeapMem += offHeapMemForExec;
+            }
+            Double cpuForExec = td.getTotalCpuReqTask(exec);
+            if (cpuForExec != null) {
+                cpu += cpuForExec;
+            }
+        }
+        slot.allocateResource(onHeapMem, offHeapMem, cpu);
+    }
+
     public void assign(WorkerSlot target, TopologyDetails td, Collection<ExecutorDetails> executors,
                        Cluster cluster) {
         if (!_isAlive) {
@@ -248,6 +275,7 @@ public class RAS_Node {
         if (!_freeSlots.contains(target)) {
             throw new IllegalStateException("Trying to assign already used slot" + target.getPort() + "on node " + _nodeId);
         } else {
+            allocateResourceToSlot(td, executors, target);
             cluster.assign(target, td.getId(), executors);
             assignInternal(target, td.getId(), false);
         }


[05/18] storm git commit: Minor Fix of the import

Posted by kn...@apache.org.
Minor Fix of the import


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

Branch: refs/heads/master
Commit: 3a09ef74e28279a81c5baeeade91a081c8fe5cf2
Parents: a77d058
Author: zhuol <zh...@yahoo-inc.com>
Authored: Thu Oct 8 10:50:27 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:38:44 2015 -0500

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/local_state.clj | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3a09ef74/storm-core/src/clj/backtype/storm/local_state.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/local_state.clj b/storm-core/src/clj/backtype/storm/local_state.clj
index a8f3334..4eaae15 100644
--- a/storm-core/src/clj/backtype/storm/local_state.clj
+++ b/storm-core/src/clj/backtype/storm/local_state.clj
@@ -19,7 +19,7 @@
             InvalidTopologyException GlobalStreamId
             LSSupervisorId LSApprovedWorkers
             LSSupervisorAssignments LocalAssignment
-            ExecutorInfo
+            ExecutorInfo LSWorkerHeartbeat
             WorkerResources])
   (:import [backtype.storm.utils LocalState]))
 


[16/18] storm git commit: Upmerge

Posted by kn...@apache.org.
Upmerge


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

Branch: refs/heads/master
Commit: b0e52ac89094fe08d725adf6a3d82374fd61c49e
Parents: 9bf8b27
Author: zhuol <zh...@yahoo-inc.com>
Authored: Mon Oct 19 09:43:32 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:44:13 2015 -0500

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/Config.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b0e52ac8/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index a1c732d..c2ed719 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1047,8 +1047,9 @@ public class Config extends HashMap<String, Object> {
     /**
      * 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";
-    public static final Object WORKER_HEAP_MEMORY_MB_SCHEMA = ConfigValidation.PositiveIntegerValidator;
 
     /**
      * The jvm opts provided to workers launched by this supervisor for GC. All "%ID%" substrings are replaced


[04/18] storm git commit: Push Thrift file changes

Posted by kn...@apache.org.
Push Thrift file changes


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

Branch: refs/heads/master
Commit: 584ded8e0f80482cf9e8f30938525fa0374490c2
Parents: 3a09ef7
Author: zhuol <zh...@yahoo-inc.com>
Authored: Thu Oct 8 16:19:49 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:38:44 2015 -0500

----------------------------------------------------------------------
 .../storm/generated/AlreadyAliveException.java  |   2 +-
 .../backtype/storm/generated/Assignment.java    | 308 ++++++++--
 .../storm/generated/AuthorizationException.java |   2 +-
 .../src/jvm/backtype/storm/generated/Bolt.java  |   2 +-
 .../storm/generated/BoltAggregateStats.java     |   2 +-
 .../jvm/backtype/storm/generated/BoltStats.java |   2 +-
 .../storm/generated/ClusterSummary.java         |   2 +-
 .../storm/generated/ClusterWorkerHeartbeat.java |  54 +-
 .../storm/generated/CommonAggregateStats.java   |   2 +-
 .../generated/ComponentAggregateStats.java      |   2 +-
 .../storm/generated/ComponentCommon.java        |   2 +-
 .../storm/generated/ComponentPageInfo.java      |   2 +-
 .../backtype/storm/generated/Credentials.java   |   2 +-
 .../storm/generated/DRPCExecutionException.java |   2 +-
 .../backtype/storm/generated/DRPCRequest.java   |   2 +-
 .../backtype/storm/generated/DebugOptions.java  |   2 +-
 .../storm/generated/DistributedRPC.java         |   2 +-
 .../generated/DistributedRPCInvocations.java    |   2 +-
 .../jvm/backtype/storm/generated/ErrorInfo.java |   2 +-
 .../storm/generated/ExecutorAggregateStats.java |   2 +-
 .../backtype/storm/generated/ExecutorInfo.java  |   2 +-
 .../backtype/storm/generated/ExecutorStats.java |   2 +-
 .../storm/generated/ExecutorSummary.java        |   2 +-
 .../storm/generated/GetInfoOptions.java         |   2 +-
 .../storm/generated/GlobalStreamId.java         |   2 +-
 .../generated/InvalidTopologyException.java     |   2 +-
 .../backtype/storm/generated/JavaObject.java    |   2 +-
 .../backtype/storm/generated/KillOptions.java   |   2 +-
 .../storm/generated/LSApprovedWorkers.java      |  46 +-
 .../generated/LSSupervisorAssignments.java      |  50 +-
 .../storm/generated/LSSupervisorId.java         |   2 +-
 .../storm/generated/LSWorkerHeartbeat.java      |  38 +-
 .../storm/generated/LocalAssignment.java        | 155 ++++-
 .../storm/generated/LocalStateData.java         |  50 +-
 .../jvm/backtype/storm/generated/LogConfig.java |  50 +-
 .../jvm/backtype/storm/generated/LogLevel.java  |   2 +-
 .../jvm/backtype/storm/generated/Nimbus.java    |   2 +-
 .../backtype/storm/generated/NimbusSummary.java |   2 +-
 .../jvm/backtype/storm/generated/NodeInfo.java  |   2 +-
 .../storm/generated/NotAliveException.java      |   2 +-
 .../backtype/storm/generated/NullStruct.java    |   2 +-
 .../storm/generated/RebalanceOptions.java       |   2 +-
 .../storm/generated/ShellComponent.java         |   2 +-
 .../storm/generated/SpoutAggregateStats.java    |   2 +-
 .../jvm/backtype/storm/generated/SpoutSpec.java |   2 +-
 .../backtype/storm/generated/SpoutStats.java    |   2 +-
 .../storm/generated/StateSpoutSpec.java         |   2 +-
 .../jvm/backtype/storm/generated/StormBase.java |  94 +--
 .../backtype/storm/generated/StormTopology.java |   2 +-
 .../backtype/storm/generated/StreamInfo.java    |   2 +-
 .../backtype/storm/generated/SubmitOptions.java |   2 +-
 .../storm/generated/SupervisorInfo.java         |   2 +-
 .../storm/generated/SupervisorSummary.java      |   2 +-
 .../storm/generated/ThriftSerializedObject.java |   2 +-
 .../backtype/storm/generated/TopologyInfo.java  |   2 +-
 .../storm/generated/TopologyPageInfo.java       |   2 +-
 .../backtype/storm/generated/TopologyStats.java |   2 +-
 .../storm/generated/TopologySummary.java        |   2 +-
 .../storm/generated/WorkerResources.java        | 605 +++++++++++++++++++
 storm-core/src/py/storm/ttypes.py               | 316 +++++++---
 60 files changed, 1447 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java b/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
index 62647c3..fb2eee3 100644
--- a/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/Assignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Assignment.java b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
index 6405c05..dbc1cc9 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Assignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -59,6 +59,7 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
   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 {
@@ -70,13 +71,15 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
   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");
+    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>();
 
@@ -99,6 +102,8 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           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;
       }
@@ -139,7 +144,7 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.NODE_HOST,_Fields.EXECUTOR_NODE_PORT,_Fields.EXECUTOR_START_TIME_SECS};
+  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);
@@ -159,6 +164,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
             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);
   }
@@ -170,6 +179,8 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
 
     this.executor_start_time_secs = new HashMap<List<Long>,Long>();
 
+    this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+
   }
 
   public Assignment(
@@ -220,6 +231,21 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       }
       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() {
@@ -235,6 +261,8 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
 
     this.executor_start_time_secs = new HashMap<List<Long>,Long>();
 
+    this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+
   }
 
   public String get_master_code_dir() {
@@ -362,6 +390,40 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
     }
   }
 
+  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:
@@ -396,6 +458,14 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       }
       break;
 
+    case WORKER_RESOURCES:
+      if (value == null) {
+        unset_worker_resources();
+      } else {
+        set_worker_resources((Map<NodeInfo,WorkerResources>)value);
+      }
+      break;
+
     }
   }
 
@@ -413,6 +483,9 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
     case EXECUTOR_START_TIME_SECS:
       return get_executor_start_time_secs();
 
+    case WORKER_RESOURCES:
+      return get_worker_resources();
+
     }
     throw new IllegalStateException();
   }
@@ -432,6 +505,8 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       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();
   }
@@ -485,6 +560,15 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
         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;
   }
 
@@ -512,6 +596,11 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
     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();
   }
 
@@ -563,6 +652,16 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
         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;
   }
 
@@ -620,6 +719,16 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       }
       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();
   }
@@ -756,6 +865,28 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
               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 _map542 = iprot.readMapBegin();
+                struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map542.size);
+                NodeInfo _key543;
+                WorkerResources _val544;
+                for (int _i545 = 0; _i545 < _map542.size; ++_i545)
+                {
+                  _key543 = new NodeInfo();
+                  _key543.read(iprot);
+                  _val544 = new WorkerResources();
+                  _val544.read(iprot);
+                  struct.worker_resources.put(_key543, _val544);
+                }
+                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);
         }
@@ -779,10 +910,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           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> _iter542 : struct.node_host.entrySet())
+            for (Map.Entry<String, String> _iter546 : struct.node_host.entrySet())
             {
-              oprot.writeString(_iter542.getKey());
-              oprot.writeString(_iter542.getValue());
+              oprot.writeString(_iter546.getKey());
+              oprot.writeString(_iter546.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -794,17 +925,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           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> _iter543 : struct.executor_node_port.entrySet())
+            for (Map.Entry<List<Long>, NodeInfo> _iter547 : struct.executor_node_port.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter543.getKey().size()));
-                for (long _iter544 : _iter543.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter547.getKey().size()));
+                for (long _iter548 : _iter547.getKey())
                 {
-                  oprot.writeI64(_iter544);
+                  oprot.writeI64(_iter548);
                 }
                 oprot.writeListEnd();
               }
-              _iter543.getValue().write(oprot);
+              _iter547.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -816,17 +947,32 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           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> _iter545 : struct.executor_start_time_secs.entrySet())
+            for (Map.Entry<List<Long>, Long> _iter549 : struct.executor_start_time_secs.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter545.getKey().size()));
-                for (long _iter546 : _iter545.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter549.getKey().size()));
+                for (long _iter550 : _iter549.getKey())
                 {
-                  oprot.writeI64(_iter546);
+                  oprot.writeI64(_iter550);
                 }
                 oprot.writeListEnd();
               }
-              oprot.writeI64(_iter545.getValue());
+              oprot.writeI64(_iter549.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> _iter551 : struct.worker_resources.entrySet())
+            {
+              _iter551.getKey().write(oprot);
+              _iter551.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -861,46 +1007,59 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       if (struct.is_set_executor_start_time_secs()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      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> _iter547 : struct.node_host.entrySet())
+          for (Map.Entry<String, String> _iter552 : struct.node_host.entrySet())
           {
-            oprot.writeString(_iter547.getKey());
-            oprot.writeString(_iter547.getValue());
+            oprot.writeString(_iter552.getKey());
+            oprot.writeString(_iter552.getValue());
           }
         }
       }
       if (struct.is_set_executor_node_port()) {
         {
           oprot.writeI32(struct.executor_node_port.size());
-          for (Map.Entry<List<Long>, NodeInfo> _iter548 : struct.executor_node_port.entrySet())
+          for (Map.Entry<List<Long>, NodeInfo> _iter553 : struct.executor_node_port.entrySet())
           {
             {
-              oprot.writeI32(_iter548.getKey().size());
-              for (long _iter549 : _iter548.getKey())
+              oprot.writeI32(_iter553.getKey().size());
+              for (long _iter554 : _iter553.getKey())
               {
-                oprot.writeI64(_iter549);
+                oprot.writeI64(_iter554);
               }
             }
-            _iter548.getValue().write(oprot);
+            _iter553.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> _iter550 : struct.executor_start_time_secs.entrySet())
+          for (Map.Entry<List<Long>, Long> _iter555 : struct.executor_start_time_secs.entrySet())
           {
             {
-              oprot.writeI32(_iter550.getKey().size());
-              for (long _iter551 : _iter550.getKey())
+              oprot.writeI32(_iter555.getKey().size());
+              for (long _iter556 : _iter555.getKey())
               {
-                oprot.writeI64(_iter551);
+                oprot.writeI64(_iter556);
               }
             }
-            oprot.writeI64(_iter550.getValue());
+            oprot.writeI64(_iter555.getValue());
+          }
+        }
+      }
+      if (struct.is_set_worker_resources()) {
+        {
+          oprot.writeI32(struct.worker_resources.size());
+          for (Map.Entry<NodeInfo, WorkerResources> _iter557 : struct.worker_resources.entrySet())
+          {
+            _iter557.getKey().write(oprot);
+            _iter557.getValue().write(oprot);
           }
         }
       }
@@ -911,71 +1070,88 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.master_code_dir = iprot.readString();
       struct.set_master_code_dir_isSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map552 = 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*_map552.size);
-          String _key553;
-          String _val554;
-          for (int _i555 = 0; _i555 < _map552.size; ++_i555)
+          org.apache.thrift.protocol.TMap _map558 = 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*_map558.size);
+          String _key559;
+          String _val560;
+          for (int _i561 = 0; _i561 < _map558.size; ++_i561)
           {
-            _key553 = iprot.readString();
-            _val554 = iprot.readString();
-            struct.node_host.put(_key553, _val554);
+            _key559 = iprot.readString();
+            _val560 = iprot.readString();
+            struct.node_host.put(_key559, _val560);
           }
         }
         struct.set_node_host_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map556 = 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*_map556.size);
-          List<Long> _key557;
-          NodeInfo _val558;
-          for (int _i559 = 0; _i559 < _map556.size; ++_i559)
+          org.apache.thrift.protocol.TMap _map562 = 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*_map562.size);
+          List<Long> _key563;
+          NodeInfo _val564;
+          for (int _i565 = 0; _i565 < _map562.size; ++_i565)
           {
             {
-              org.apache.thrift.protocol.TList _list560 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key557 = new ArrayList<Long>(_list560.size);
-              long _elem561;
-              for (int _i562 = 0; _i562 < _list560.size; ++_i562)
+              org.apache.thrift.protocol.TList _list566 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key563 = new ArrayList<Long>(_list566.size);
+              long _elem567;
+              for (int _i568 = 0; _i568 < _list566.size; ++_i568)
               {
-                _elem561 = iprot.readI64();
-                _key557.add(_elem561);
+                _elem567 = iprot.readI64();
+                _key563.add(_elem567);
               }
             }
-            _val558 = new NodeInfo();
-            _val558.read(iprot);
-            struct.executor_node_port.put(_key557, _val558);
+            _val564 = new NodeInfo();
+            _val564.read(iprot);
+            struct.executor_node_port.put(_key563, _val564);
           }
         }
         struct.set_executor_node_port_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map563 = 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*_map563.size);
-          List<Long> _key564;
-          long _val565;
-          for (int _i566 = 0; _i566 < _map563.size; ++_i566)
+          org.apache.thrift.protocol.TMap _map569 = 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*_map569.size);
+          List<Long> _key570;
+          long _val571;
+          for (int _i572 = 0; _i572 < _map569.size; ++_i572)
           {
             {
-              org.apache.thrift.protocol.TList _list567 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key564 = new ArrayList<Long>(_list567.size);
-              long _elem568;
-              for (int _i569 = 0; _i569 < _list567.size; ++_i569)
+              org.apache.thrift.protocol.TList _list573 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key570 = new ArrayList<Long>(_list573.size);
+              long _elem574;
+              for (int _i575 = 0; _i575 < _list573.size; ++_i575)
               {
-                _elem568 = iprot.readI64();
-                _key564.add(_elem568);
+                _elem574 = iprot.readI64();
+                _key570.add(_elem574);
               }
             }
-            _val565 = iprot.readI64();
-            struct.executor_start_time_secs.put(_key564, _val565);
+            _val571 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key570, _val571);
           }
         }
         struct.set_executor_start_time_secs_isSet(true);
       }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TMap _map576 = 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*_map576.size);
+          NodeInfo _key577;
+          WorkerResources _val578;
+          for (int _i579 = 0; _i579 < _map576.size; ++_i579)
+          {
+            _key577 = new NodeInfo();
+            _key577.read(iprot);
+            _val578 = new WorkerResources();
+            _val578.read(iprot);
+            struct.worker_resources.put(_key577, _val578);
+          }
+        }
+        struct.set_worker_resources_isSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
index a179751..69fff12 100644
--- a/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/Bolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Bolt.java b/storm-core/src/jvm/backtype/storm/generated/Bolt.java
index 861226c..9ea4bef 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Bolt.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Bolt.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
index 0ddcaba..940e0b9 100644
--- a/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/BoltStats.java b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
index 315f955..c3ffc9f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
index 57543db..d23cdcb 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
index 0b79fc3..6c84d4c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -635,17 +635,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
           case 2: // EXECUTOR_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map590 = iprot.readMapBegin();
-                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map590.size);
-                ExecutorInfo _key591;
-                ExecutorStats _val592;
-                for (int _i593 = 0; _i593 < _map590.size; ++_i593)
+                org.apache.thrift.protocol.TMap _map600 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map600.size);
+                ExecutorInfo _key601;
+                ExecutorStats _val602;
+                for (int _i603 = 0; _i603 < _map600.size; ++_i603)
                 {
-                  _key591 = new ExecutorInfo();
-                  _key591.read(iprot);
-                  _val592 = new ExecutorStats();
-                  _val592.read(iprot);
-                  struct.executor_stats.put(_key591, _val592);
+                  _key601 = new ExecutorInfo();
+                  _key601.read(iprot);
+                  _val602 = new ExecutorStats();
+                  _val602.read(iprot);
+                  struct.executor_stats.put(_key601, _val602);
                 }
                 iprot.readMapEnd();
               }
@@ -692,10 +692,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
         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> _iter594 : struct.executor_stats.entrySet())
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter604 : struct.executor_stats.entrySet())
           {
-            _iter594.getKey().write(oprot);
-            _iter594.getValue().write(oprot);
+            _iter604.getKey().write(oprot);
+            _iter604.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -727,10 +727,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       oprot.writeString(struct.storm_id);
       {
         oprot.writeI32(struct.executor_stats.size());
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter595 : struct.executor_stats.entrySet())
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter605 : struct.executor_stats.entrySet())
         {
-          _iter595.getKey().write(oprot);
-          _iter595.getValue().write(oprot);
+          _iter605.getKey().write(oprot);
+          _iter605.getValue().write(oprot);
         }
       }
       oprot.writeI32(struct.time_secs);
@@ -743,17 +743,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       struct.storm_id = iprot.readString();
       struct.set_storm_id_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map596 = 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*_map596.size);
-        ExecutorInfo _key597;
-        ExecutorStats _val598;
-        for (int _i599 = 0; _i599 < _map596.size; ++_i599)
+        org.apache.thrift.protocol.TMap _map606 = 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*_map606.size);
+        ExecutorInfo _key607;
+        ExecutorStats _val608;
+        for (int _i609 = 0; _i609 < _map606.size; ++_i609)
         {
-          _key597 = new ExecutorInfo();
-          _key597.read(iprot);
-          _val598 = new ExecutorStats();
-          _val598.read(iprot);
-          struct.executor_stats.put(_key597, _val598);
+          _key607 = new ExecutorInfo();
+          _key607.read(iprot);
+          _val608 = new ExecutorStats();
+          _val608.read(iprot);
+          struct.executor_stats.put(_key607, _val608);
         }
       }
       struct.set_executor_stats_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
index 5aa1c6e..108f74f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
index 0f7336a..8fe77b3 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java b/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
index 6868603..c69edf2 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
index b6b6862..c8fcf7b 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/Credentials.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Credentials.java b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
index afd9e2f..71a764f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Credentials.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java b/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
index f61ffa1..1471537 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java b/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
index b97fa25..4648c36 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
index 60c0e24..da7a45a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
index c4f25ec..475746b 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 public class DistributedRPC {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
index 94b799b..d0c7f64 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 public class DistributedRPCInvocations {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java b/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
index 32495c7..515a436 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
index b07febf..4993cc5 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
index 44415ce..3f27a51 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
index 1c78f2d..4783c82 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
index 649342d..4f70a32 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java b/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java
index 2d5defc..90c811f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java b/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
index 017ad4f..8a0e1c7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
+++ b/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java b/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
index 5a98418..003dae7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/JavaObject.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/JavaObject.java b/storm-core/src/jvm/backtype/storm/generated/JavaObject.java
index fde0082..15d8fd7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/JavaObject.java
+++ b/storm-core/src/jvm/backtype/storm/generated/JavaObject.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/KillOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/KillOptions.java b/storm-core/src/jvm/backtype/storm/generated/KillOptions.java
index f173395..bebfbb1 100644
--- a/storm-core/src/jvm/backtype/storm/generated/KillOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/KillOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
index a38724d..75da228 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -365,15 +365,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
           case 1: // APPROVED_WORKERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map618 = iprot.readMapBegin();
-                struct.approved_workers = new HashMap<String,Integer>(2*_map618.size);
-                String _key619;
-                int _val620;
-                for (int _i621 = 0; _i621 < _map618.size; ++_i621)
+                org.apache.thrift.protocol.TMap _map628 = iprot.readMapBegin();
+                struct.approved_workers = new HashMap<String,Integer>(2*_map628.size);
+                String _key629;
+                int _val630;
+                for (int _i631 = 0; _i631 < _map628.size; ++_i631)
                 {
-                  _key619 = iprot.readString();
-                  _val620 = iprot.readI32();
-                  struct.approved_workers.put(_key619, _val620);
+                  _key629 = iprot.readString();
+                  _val630 = iprot.readI32();
+                  struct.approved_workers.put(_key629, _val630);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
         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> _iter622 : struct.approved_workers.entrySet())
+          for (Map.Entry<String, Integer> _iter632 : struct.approved_workers.entrySet())
           {
-            oprot.writeString(_iter622.getKey());
-            oprot.writeI32(_iter622.getValue());
+            oprot.writeString(_iter632.getKey());
+            oprot.writeI32(_iter632.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.approved_workers.size());
-        for (Map.Entry<String, Integer> _iter623 : struct.approved_workers.entrySet())
+        for (Map.Entry<String, Integer> _iter633 : struct.approved_workers.entrySet())
         {
-          oprot.writeString(_iter623.getKey());
-          oprot.writeI32(_iter623.getValue());
+          oprot.writeString(_iter633.getKey());
+          oprot.writeI32(_iter633.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
     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 _map624 = 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*_map624.size);
-        String _key625;
-        int _val626;
-        for (int _i627 = 0; _i627 < _map624.size; ++_i627)
+        org.apache.thrift.protocol.TMap _map634 = 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*_map634.size);
+        String _key635;
+        int _val636;
+        for (int _i637 = 0; _i637 < _map634.size; ++_i637)
         {
-          _key625 = iprot.readString();
-          _val626 = iprot.readI32();
-          struct.approved_workers.put(_key625, _val626);
+          _key635 = iprot.readString();
+          _val636 = iprot.readI32();
+          struct.approved_workers.put(_key635, _val636);
         }
       }
       struct.set_approved_workers_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
index 82a9fad..7eb3628 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -376,16 +376,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
           case 1: // ASSIGNMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map628 = iprot.readMapBegin();
-                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map628.size);
-                int _key629;
-                LocalAssignment _val630;
-                for (int _i631 = 0; _i631 < _map628.size; ++_i631)
+                org.apache.thrift.protocol.TMap _map638 = iprot.readMapBegin();
+                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map638.size);
+                int _key639;
+                LocalAssignment _val640;
+                for (int _i641 = 0; _i641 < _map638.size; ++_i641)
                 {
-                  _key629 = iprot.readI32();
-                  _val630 = new LocalAssignment();
-                  _val630.read(iprot);
-                  struct.assignments.put(_key629, _val630);
+                  _key639 = iprot.readI32();
+                  _val640 = new LocalAssignment();
+                  _val640.read(iprot);
+                  struct.assignments.put(_key639, _val640);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
         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> _iter632 : struct.assignments.entrySet())
+          for (Map.Entry<Integer, LocalAssignment> _iter642 : struct.assignments.entrySet())
           {
-            oprot.writeI32(_iter632.getKey());
-            _iter632.getValue().write(oprot);
+            oprot.writeI32(_iter642.getKey());
+            _iter642.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.assignments.size());
-        for (Map.Entry<Integer, LocalAssignment> _iter633 : struct.assignments.entrySet())
+        for (Map.Entry<Integer, LocalAssignment> _iter643 : struct.assignments.entrySet())
         {
-          oprot.writeI32(_iter633.getKey());
-          _iter633.getValue().write(oprot);
+          oprot.writeI32(_iter643.getKey());
+          _iter643.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
     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 _map634 = 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*_map634.size);
-        int _key635;
-        LocalAssignment _val636;
-        for (int _i637 = 0; _i637 < _map634.size; ++_i637)
+        org.apache.thrift.protocol.TMap _map644 = 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*_map644.size);
+        int _key645;
+        LocalAssignment _val646;
+        for (int _i647 = 0; _i647 < _map644.size; ++_i647)
         {
-          _key635 = iprot.readI32();
-          _val636 = new LocalAssignment();
-          _val636.read(iprot);
-          struct.assignments.put(_key635, _val636);
+          _key645 = iprot.readI32();
+          _val646 = new LocalAssignment();
+          _val646.read(iprot);
+          struct.assignments.put(_key645, _val646);
         }
       }
       struct.set_assignments_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java
index 6ee4dad..8b0241a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
index 053707d..fc37eb1 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -638,14 +638,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
           case 3: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list638 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list638.size);
-                ExecutorInfo _elem639;
-                for (int _i640 = 0; _i640 < _list638.size; ++_i640)
+                org.apache.thrift.protocol.TList _list648 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list648.size);
+                ExecutorInfo _elem649;
+                for (int _i650 = 0; _i650 < _list648.size; ++_i650)
                 {
-                  _elem639 = new ExecutorInfo();
-                  _elem639.read(iprot);
-                  struct.executors.add(_elem639);
+                  _elem649 = new ExecutorInfo();
+                  _elem649.read(iprot);
+                  struct.executors.add(_elem649);
                 }
                 iprot.readListEnd();
               }
@@ -687,9 +687,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter641 : struct.executors)
+          for (ExecutorInfo _iter651 : struct.executors)
           {
-            _iter641.write(oprot);
+            _iter651.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter642 : struct.executors)
+        for (ExecutorInfo _iter652 : struct.executors)
         {
-          _iter642.write(oprot);
+          _iter652.write(oprot);
         }
       }
       oprot.writeI32(struct.port);
@@ -735,14 +735,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list643.size);
-        ExecutorInfo _elem644;
-        for (int _i645 = 0; _i645 < _list643.size; ++_i645)
+        org.apache.thrift.protocol.TList _list653 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list653.size);
+        ExecutorInfo _elem654;
+        for (int _i655 = 0; _i655 < _list653.size; ++_i655)
         {
-          _elem644 = new ExecutorInfo();
-          _elem644.read(iprot);
-          struct.executors.add(_elem644);
+          _elem654 = new ExecutorInfo();
+          _elem654.read(iprot);
+          struct.executors.add(_elem654);
         }
       }
       struct.set_executors_isSet(true);


[03/18] storm git commit: Push Thrift file changes

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
index f7b034f..1e9f05c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
@@ -51,12 +51,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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 {
@@ -66,11 +67,13 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
 
   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");
+    EXECUTORS((short)2, "executors"),
+    RESOURCES((short)3, "resources");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -89,6 +92,8 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
           return TOPOLOGY_ID;
         case 2: // EXECUTORS
           return EXECUTORS;
+        case 3: // RESOURCES
+          return RESOURCES;
         default:
           return null;
       }
@@ -129,6 +134,7 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
   }
 
   // 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);
@@ -137,6 +143,8 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
     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);
   }
@@ -167,6 +175,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       }
       this.executors = __this__executors;
     }
+    if (other.is_set_resources()) {
+      this.resources = new WorkerResources(other.resources);
+    }
   }
 
   public LocalAssignment deepCopy() {
@@ -177,6 +188,7 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
   public void clear() {
     this.topology_id = null;
     this.executors = null;
+    this.resources = null;
   }
 
   public String get_topology_id() {
@@ -240,6 +252,29 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
     }
   }
 
+  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:
@@ -258,6 +293,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       }
       break;
 
+    case RESOURCES:
+      if (value == null) {
+        unset_resources();
+      } else {
+        set_resources((WorkerResources)value);
+      }
+      break;
+
     }
   }
 
@@ -269,6 +312,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
     case EXECUTORS:
       return get_executors();
 
+    case RESOURCES:
+      return get_resources();
+
     }
     throw new IllegalStateException();
   }
@@ -284,6 +330,8 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       return is_set_topology_id();
     case EXECUTORS:
       return is_set_executors();
+    case RESOURCES:
+      return is_set_resources();
     }
     throw new IllegalStateException();
   }
@@ -319,6 +367,15 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
         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;
   }
 
@@ -336,6 +393,11 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
     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();
   }
 
@@ -367,6 +429,16 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
         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;
   }
 
@@ -402,6 +474,16 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       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();
   }
@@ -417,6 +499,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
     }
 
     // check for sub-struct validity
+    if (resources != null) {
+      resources.validate();
+    }
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -464,14 +549,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
           case 2: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list610.size);
-                ExecutorInfo _elem611;
-                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+                org.apache.thrift.protocol.TList _list620 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list620.size);
+                ExecutorInfo _elem621;
+                for (int _i622 = 0; _i622 < _list620.size; ++_i622)
                 {
-                  _elem611 = new ExecutorInfo();
-                  _elem611.read(iprot);
-                  struct.executors.add(_elem611);
+                  _elem621 = new ExecutorInfo();
+                  _elem621.read(iprot);
+                  struct.executors.add(_elem621);
                 }
                 iprot.readListEnd();
               }
@@ -480,6 +565,15 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
               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);
         }
@@ -502,14 +596,21 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter613 : struct.executors)
+          for (ExecutorInfo _iter623 : struct.executors)
           {
-            _iter613.write(oprot);
+            _iter623.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();
     }
@@ -530,11 +631,19 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter614 : struct.executors)
+        for (ExecutorInfo _iter624 : struct.executors)
         {
-          _iter614.write(oprot);
+          _iter624.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
@@ -543,17 +652,23 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list615.size);
-        ExecutorInfo _elem616;
-        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
+        org.apache.thrift.protocol.TList _list625 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list625.size);
+        ExecutorInfo _elem626;
+        for (int _i627 = 0; _i627 < _list625.size; ++_i627)
         {
-          _elem616 = new ExecutorInfo();
-          _elem616.read(iprot);
-          struct.executors.add(_elem616);
+          _elem626 = new ExecutorInfo();
+          _elem626.read(iprot);
+          struct.executors.add(_elem626);
         }
       }
       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/584ded8e/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
index eb1ca97..e821149 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -376,16 +376,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
           case 1: // SERIALIZED_PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map600 = iprot.readMapBegin();
-                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map600.size);
-                String _key601;
-                ThriftSerializedObject _val602;
-                for (int _i603 = 0; _i603 < _map600.size; ++_i603)
+                org.apache.thrift.protocol.TMap _map610 = iprot.readMapBegin();
+                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map610.size);
+                String _key611;
+                ThriftSerializedObject _val612;
+                for (int _i613 = 0; _i613 < _map610.size; ++_i613)
                 {
-                  _key601 = iprot.readString();
-                  _val602 = new ThriftSerializedObject();
-                  _val602.read(iprot);
-                  struct.serialized_parts.put(_key601, _val602);
+                  _key611 = iprot.readString();
+                  _val612 = new ThriftSerializedObject();
+                  _val612.read(iprot);
+                  struct.serialized_parts.put(_key611, _val612);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
         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> _iter604 : struct.serialized_parts.entrySet())
+          for (Map.Entry<String, ThriftSerializedObject> _iter614 : struct.serialized_parts.entrySet())
           {
-            oprot.writeString(_iter604.getKey());
-            _iter604.getValue().write(oprot);
+            oprot.writeString(_iter614.getKey());
+            _iter614.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.serialized_parts.size());
-        for (Map.Entry<String, ThriftSerializedObject> _iter605 : struct.serialized_parts.entrySet())
+        for (Map.Entry<String, ThriftSerializedObject> _iter615 : struct.serialized_parts.entrySet())
         {
-          oprot.writeString(_iter605.getKey());
-          _iter605.getValue().write(oprot);
+          oprot.writeString(_iter615.getKey());
+          _iter615.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
     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 _map606 = 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*_map606.size);
-        String _key607;
-        ThriftSerializedObject _val608;
-        for (int _i609 = 0; _i609 < _map606.size; ++_i609)
+        org.apache.thrift.protocol.TMap _map616 = 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*_map616.size);
+        String _key617;
+        ThriftSerializedObject _val618;
+        for (int _i619 = 0; _i619 < _map616.size; ++_i619)
         {
-          _key607 = iprot.readString();
-          _val608 = new ThriftSerializedObject();
-          _val608.read(iprot);
-          struct.serialized_parts.put(_key607, _val608);
+          _key617 = iprot.readString();
+          _val618 = new ThriftSerializedObject();
+          _val618.read(iprot);
+          struct.serialized_parts.put(_key617, _val618);
         }
       }
       struct.set_serialized_parts_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
index b9bfae2..6d537f8 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -368,16 +368,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           case 2: // NAMED_LOGGER_LEVEL
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map646 = iprot.readMapBegin();
-                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map646.size);
-                String _key647;
-                LogLevel _val648;
-                for (int _i649 = 0; _i649 < _map646.size; ++_i649)
+                org.apache.thrift.protocol.TMap _map656 = iprot.readMapBegin();
+                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map656.size);
+                String _key657;
+                LogLevel _val658;
+                for (int _i659 = 0; _i659 < _map656.size; ++_i659)
                 {
-                  _key647 = iprot.readString();
-                  _val648 = new LogLevel();
-                  _val648.read(iprot);
-                  struct.named_logger_level.put(_key647, _val648);
+                  _key657 = iprot.readString();
+                  _val658 = new LogLevel();
+                  _val658.read(iprot);
+                  struct.named_logger_level.put(_key657, _val658);
                 }
                 iprot.readMapEnd();
               }
@@ -404,10 +404,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           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> _iter650 : struct.named_logger_level.entrySet())
+            for (Map.Entry<String, LogLevel> _iter660 : struct.named_logger_level.entrySet())
             {
-              oprot.writeString(_iter650.getKey());
-              _iter650.getValue().write(oprot);
+              oprot.writeString(_iter660.getKey());
+              _iter660.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -439,10 +439,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       if (struct.is_set_named_logger_level()) {
         {
           oprot.writeI32(struct.named_logger_level.size());
-          for (Map.Entry<String, LogLevel> _iter651 : struct.named_logger_level.entrySet())
+          for (Map.Entry<String, LogLevel> _iter661 : struct.named_logger_level.entrySet())
           {
-            oprot.writeString(_iter651.getKey());
-            _iter651.getValue().write(oprot);
+            oprot.writeString(_iter661.getKey());
+            _iter661.getValue().write(oprot);
           }
         }
       }
@@ -454,16 +454,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       BitSet incoming = iprot.readBitSet(1);
       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.STRUCT, iprot.readI32());
-          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map652.size);
-          String _key653;
-          LogLevel _val654;
-          for (int _i655 = 0; _i655 < _map652.size; ++_i655)
+          org.apache.thrift.protocol.TMap _map662 = 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*_map662.size);
+          String _key663;
+          LogLevel _val664;
+          for (int _i665 = 0; _i665 < _map662.size; ++_i665)
           {
-            _key653 = iprot.readString();
-            _val654 = new LogLevel();
-            _val654.read(iprot);
-            struct.named_logger_level.put(_key653, _val654);
+            _key663 = iprot.readString();
+            _val664 = new LogLevel();
+            _val664.read(iprot);
+            struct.named_logger_level.put(_key663, _val664);
           }
         }
         struct.set_named_logger_level_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogLevel.java b/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
index becf92c..1526bf3 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index 9e28e38..59f74fb 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 public class Nimbus {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
index 1d0f3fb..7d1e2fd 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-2")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
index a2b5c1e..364afa7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java b/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java
index 100da84..cbabcf9 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/NullStruct.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NullStruct.java b/storm-core/src/jvm/backtype/storm/generated/NullStruct.java
index f44bccd..1b8208c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NullStruct.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NullStruct.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
index d49d4a9..d859f5a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java b/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java
index 27303f1..ab86c6a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
index a42f434..a8d6ec7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java b/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java
index 42fd37b..3fc45cf 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
index 2c39746..478143f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java b/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java
index 15d34ae..530b7ca 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
index f7ad22c..f4af67a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StormBase.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 
@@ -1090,15 +1090,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 4: // COMPONENT_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map570 = iprot.readMapBegin();
-                struct.component_executors = new HashMap<String,Integer>(2*_map570.size);
-                String _key571;
-                int _val572;
-                for (int _i573 = 0; _i573 < _map570.size; ++_i573)
+                org.apache.thrift.protocol.TMap _map580 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map580.size);
+                String _key581;
+                int _val582;
+                for (int _i583 = 0; _i583 < _map580.size; ++_i583)
                 {
-                  _key571 = iprot.readString();
-                  _val572 = iprot.readI32();
-                  struct.component_executors.put(_key571, _val572);
+                  _key581 = iprot.readString();
+                  _val582 = iprot.readI32();
+                  struct.component_executors.put(_key581, _val582);
                 }
                 iprot.readMapEnd();
               }
@@ -1143,16 +1143,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 9: // COMPONENT_DEBUG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map574 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,DebugOptions>(2*_map574.size);
-                String _key575;
-                DebugOptions _val576;
-                for (int _i577 = 0; _i577 < _map574.size; ++_i577)
+                org.apache.thrift.protocol.TMap _map584 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map584.size);
+                String _key585;
+                DebugOptions _val586;
+                for (int _i587 = 0; _i587 < _map584.size; ++_i587)
                 {
-                  _key575 = iprot.readString();
-                  _val576 = new DebugOptions();
-                  _val576.read(iprot);
-                  struct.component_debug.put(_key575, _val576);
+                  _key585 = iprot.readString();
+                  _val586 = new DebugOptions();
+                  _val586.read(iprot);
+                  struct.component_debug.put(_key585, _val586);
                 }
                 iprot.readMapEnd();
               }
@@ -1192,10 +1192,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           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> _iter578 : struct.component_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter588 : struct.component_executors.entrySet())
             {
-              oprot.writeString(_iter578.getKey());
-              oprot.writeI32(_iter578.getValue());
+              oprot.writeString(_iter588.getKey());
+              oprot.writeI32(_iter588.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1233,10 +1233,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           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> _iter579 : struct.component_debug.entrySet())
+            for (Map.Entry<String, DebugOptions> _iter589 : struct.component_debug.entrySet())
             {
-              oprot.writeString(_iter579.getKey());
-              _iter579.getValue().write(oprot);
+              oprot.writeString(_iter589.getKey());
+              _iter589.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1286,10 +1286,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_executors()) {
         {
           oprot.writeI32(struct.component_executors.size());
-          for (Map.Entry<String, Integer> _iter580 : struct.component_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter590 : struct.component_executors.entrySet())
           {
-            oprot.writeString(_iter580.getKey());
-            oprot.writeI32(_iter580.getValue());
+            oprot.writeString(_iter590.getKey());
+            oprot.writeI32(_iter590.getValue());
           }
         }
       }
@@ -1308,10 +1308,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, DebugOptions> _iter581 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter591 : struct.component_debug.entrySet())
           {
-            oprot.writeString(_iter581.getKey());
-            _iter581.getValue().write(oprot);
+            oprot.writeString(_iter591.getKey());
+            _iter591.getValue().write(oprot);
           }
         }
       }
@@ -1329,15 +1329,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map582 = 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*_map582.size);
-          String _key583;
-          int _val584;
-          for (int _i585 = 0; _i585 < _map582.size; ++_i585)
+          org.apache.thrift.protocol.TMap _map592 = 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*_map592.size);
+          String _key593;
+          int _val594;
+          for (int _i595 = 0; _i595 < _map592.size; ++_i595)
           {
-            _key583 = iprot.readString();
-            _val584 = iprot.readI32();
-            struct.component_executors.put(_key583, _val584);
+            _key593 = iprot.readString();
+            _val594 = iprot.readI32();
+            struct.component_executors.put(_key593, _val594);
           }
         }
         struct.set_component_executors_isSet(true);
@@ -1361,16 +1361,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map586 = 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*_map586.size);
-          String _key587;
-          DebugOptions _val588;
-          for (int _i589 = 0; _i589 < _map586.size; ++_i589)
+          org.apache.thrift.protocol.TMap _map596 = 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*_map596.size);
+          String _key597;
+          DebugOptions _val598;
+          for (int _i599 = 0; _i599 < _map596.size; ++_i599)
           {
-            _key587 = iprot.readString();
-            _val588 = new DebugOptions();
-            _val588.read(iprot);
-            struct.component_debug.put(_key587, _val588);
+            _key597 = iprot.readString();
+            _val598 = new DebugOptions();
+            _val598.read(iprot);
+            struct.component_debug.put(_key597, _val598);
           }
         }
         struct.set_component_debug_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/StormTopology.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormTopology.java b/storm-core/src/jvm/backtype/storm/generated/StormTopology.java
index d022e95..9b96fa3 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StormTopology.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormTopology.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java b/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java
index b9bc75d..e3b0fdb 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
index 2a68c3b..358468a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
index 6dc35be..6d68927 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
index ed626b5..7e36d0f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java b/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java
index 6810669..4b2bc63 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index f023f63..4f78417 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
index 3528ca2..180b608 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
index b2f5b23..0ff01de 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
index 4e6ff61..055a01a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-9")
 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");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java b/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
new file mode 100644
index 0000000..7cfadd7
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-10-9")
+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 Double.valueOf(get_mem_on_heap());
+
+    case MEM_OFF_HEAP:
+      return Double.valueOf(get_mem_off_heap());
+
+    case CPU:
+      return Double.valueOf(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);
+      }
+    }
+  }
+
+}
+


[14/18] storm git commit: Fix tests due to upmerge

Posted by kn...@apache.org.
Fix tests due to upmerge


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

Branch: refs/heads/master
Commit: 9bf8b275d258300e498b5ba514cdf1e016ffde4e
Parents: e50bed8
Author: zhuol <zh...@yahoo-inc.com>
Authored: Mon Oct 12 13:36:40 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:44:12 2015 -0500

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/converter.clj        | 2 +-
 storm-core/test/clj/backtype/storm/supervisor_test.clj | 7 ++++---
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9bf8b275/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index 4d56900..21920e4 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -83,7 +83,7 @@
 
 (defn clojurify-worker->resources [worker->resources]
   "convert worker info to be [node, port]
-   convert resources to be mem_on_heap mem_off_heap cpu]"
+   convert resources to be [mem_on_heap mem_off_heap cpu]"
   (into {} (map
              (fn [[nodeInfo resources]]
                [(concat [(.get_node nodeInfo)] (.get_port nodeInfo))

http://git-wip-us.apache.org/repos/asf/storm/blob/9bf8b275/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index 907274e..301bb82 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -599,9 +599,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
-          childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m"]
-          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          mem-onheap 512
+          childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
+          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-topology-id-alone


[13/18] storm git commit: Remove generated files

Posted by kn...@apache.org.
Remove generated files

Cherry-pick YSTORM-1546

Cherry-pick YSTORM-2146


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

Branch: refs/heads/master
Commit: a77d0581a061efbe26600d50001c8f8c7ee866e3
Parents: 17d70d3
Author: zhuol <zh...@yahoo-inc.com>
Authored: Wed Oct 7 16:48:33 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:38:44 2015 -0500

----------------------------------------------------------------------
 conf/defaults.yaml                              |   3 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  11 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  31 +-
 .../src/clj/backtype/storm/local_state.clj      |   9 +-
 storm-core/src/clj/backtype/storm/testing.clj   |   2 +-
 storm-core/src/jvm/backtype/storm/Config.java   |  12 +-
 .../backtype/storm/generated/AccessControl.java | 627 ---------------
 .../generated/HBAuthorizationException.java     | 406 ----------
 .../storm/generated/HBExecutionException.java   | 406 ----------
 .../jvm/backtype/storm/generated/HBMessage.java | 636 ---------------
 .../jvm/backtype/storm/generated/HBNodes.java   | 461 -----------
 .../jvm/backtype/storm/generated/HBPulse.java   | 522 ------------
 .../jvm/backtype/storm/generated/HBRecords.java | 466 -----------
 .../generated/KeyAlreadyExistsException.java    | 406 ----------
 .../storm/generated/KeyNotFoundException.java   | 406 ----------
 .../backtype/storm/generated/LSTopoHistory.java | 805 -------------------
 .../storm/generated/LSTopoHistoryList.java      | 460 -----------
 .../storm/generated/ListBlobsResult.java        | 556 -------------
 .../storm/generated/ReadableBlobMeta.java       | 510 ------------
 .../storm/generated/SettableBlobMeta.java       | 460 -----------
 .../storm/generated/TopologyHistoryInfo.java    | 461 -----------
 .../storm/generated/WorkerResources.java        | 605 --------------
 .../backtype/storm/scheduler/WorkerSlot.java    |   3 +-
 .../test/clj/backtype/storm/supervisor_test.clj |  46 +-
 24 files changed, 78 insertions(+), 8232 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 881f8fb..dae8773 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -140,7 +140,8 @@ supervisor.memory.capacity.mb: 3072.0
 supervisor.cpu.capacity: 400.0
 
 ### worker.* configs are for task workers
-worker.childopts: "-Xmx768m"
+worker.heap.memory.mb: 768
+worker.childopts: "-Xmx%HEAP-MEM%m"
 worker.gc.childopts: ""
 worker.heartbeat.frequency.secs: 1
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 4c3d1c1..2b0a9b5 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -545,12 +545,21 @@
   (into {} (for [[tid assignment] existing-assignments
                  :let [alive-executors (topology->alive-executors tid)
                        executor->node+port (:executor->node+port assignment)
+                       worker->resources (:worker->resources assignment)
+                       ;; making a map from node+port to WorkerSlot with allocated resources
+                       node+port->slot (into {} (for [[[node port] [mem-on-heap mem-off-heap cpu]] worker->resources]
+                                                  {[node port]
+                                                   (doto (WorkerSlot. node port)
+                                                     (.allocateResource
+                                                       mem-on-heap
+                                                       mem-off-heap
+                                                       cpu))}))
                        executor->slot (into {} (for [[executor [node port]] executor->node+port]
                                                  ;; filter out the dead executors
                                                  (if (contains? alive-executors executor)
                                                    {(ExecutorDetails. (first executor)
                                                                       (second executor))
-                                                    (WorkerSlot. node port)}
+                                                    (get node+port->slot [node port])}
                                                    {})))]]
              {tid (SchedulerAssignmentImpl. tid executor->slot)})))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index d87cc90..8a844f2 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -26,6 +26,7 @@
   (:use [backtype.storm config util log timer local-state])
   (:import [backtype.storm.utils VersionInfo])
   (:import [backtype.storm Config])
+  (:import [backtype.storm.generated WorkerResources])
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm.daemon [worker :as worker]]
             [backtype.storm [process-simulator :as psim] [cluster :as cluster] [event :as event]]
@@ -78,7 +79,7 @@
     (into {} (for [[port executors] port-executors]
                ;; need to cast to int b/c it might be a long (due to how yaml parses things)
                ;; doall is to avoid serialization/deserialization problems with lazy seqs
-               [(Integer. port) (mk-local-assignment-with-resources storm-id (doall executors) (get my-slots-resources [assignment-id port]))]
+               [(Integer. port) (mk-local-assignment storm-id (doall executors) (get my-slots-resources [assignment-id port]))]
                ))))
 
 (defn- read-assignments
@@ -390,7 +391,10 @@
     (wait-for-workers-launch
      conf
      (dofor [[port assignment] reassign-executors]
-            (let [id (new-worker-ids port)]
+            (let [id (new-worker-ids port)
+                  storm-id (:storm-id assignment)
+                  ^WorkerResources resources (:resources assignment)
+                  mem-onheap (.get_mem_on_heap resources)]
               (try
                 (log-message "Launching worker with assignment "
                              (pr-str assignment)
@@ -404,7 +408,8 @@
                 (launch-worker supervisor
                                (:storm-id assignment)
                                port
-                               id)
+                               id
+                               mem-onheap)
                 (catch java.io.FileNotFoundException e
                   (log-message "Unable to launch worker due to "
                                (.getMessage e)))
@@ -641,12 +646,13 @@
     (str arch-resource-root File/pathSeparator resource-root File/pathSeparator (conf JAVA-LIBRARY-PATH))))
 
 (defn substitute-childopts
-  "Generates runtime childopts by replacing keys with topology-id, worker-id, port"
-  [value worker-id topology-id port]
+  "Generates runtime childopts by replacing keys with topology-id, worker-id, port, mem-onheap"
+  [value worker-id topology-id port mem-onheap]
   (let [replacement-map {"%ID%"          (str port)
                          "%WORKER-ID%"   (str worker-id)
                          "%TOPOLOGY-ID%"    (str topology-id)
-                         "%WORKER-PORT%" (str port)}
+                         "%WORKER-PORT%" (str port)
+                         "%HEAP-MEM%" (str mem-onheap)}
         sub-fn #(reduce (fn [string entry]
                           (apply clojure.string/replace string entry))
                         %
@@ -665,7 +671,7 @@
 
 
 (defmethod launch-worker
-    :distributed [supervisor storm-id port worker-id]
+    :distributed [supervisor storm-id port worker-id mem-onheap]
     (let [conf (:conf supervisor)
           run-worker-as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
           storm-home (System/getProperty "storm.home")
@@ -689,15 +695,18 @@
                         (add-to-classpath [stormjar])
                         (add-to-classpath topo-classpath))
           top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
-          gc-opts (substitute-childopts (if top-gc-opts top-gc-opts (conf WORKER-GC-CHILDOPTS)) worker-id storm-id port)
+          mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
+                       (int (Math/ceil mem-onheap)) ;; round up
+                       (storm-conf WORKER-HEAP-MEMORY-MB)) ;; otherwise use default value
+          gc-opts (substitute-childopts (if top-gc-opts top-gc-opts (conf WORKER-GC-CHILDOPTS)) worker-id storm-id port mem-onheap)
           topo-worker-logwriter-childopts (storm-conf TOPOLOGY-WORKER-LOGWRITER-CHILDOPTS)
           user (storm-conf TOPOLOGY-SUBMITTER-USER)
           logging-sensitivity (storm-conf TOPOLOGY-LOGGING-SENSITIVITY "S3")
           logfilename (logs-filename storm-id port)
           worker-childopts (when-let [s (conf WORKER-CHILDOPTS)]
-                             (substitute-childopts s worker-id storm-id port))
+                             (substitute-childopts s worker-id storm-id port mem-onheap))
           topo-worker-childopts (when-let [s (storm-conf TOPOLOGY-WORKER-CHILDOPTS)]
-                                  (substitute-childopts s worker-id storm-id port))
+                                  (substitute-childopts s worker-id storm-id port mem-onheap))
           topology-worker-environment (if-let [env (storm-conf TOPOLOGY-ENVIRONMENT)]
                                         (merge env {"LD_LIBRARY_PATH" jlp})
                                         {"LD_LIBRARY_PATH" jlp})
@@ -786,7 +795,7 @@
 (defmethod mk-code-distributor :local [conf] nil)
 
 (defmethod launch-worker
-    :local [supervisor storm-id port worker-id]
+    :local [supervisor storm-id port worker-id mem-onheap]
     (let [conf (:conf supervisor)
           pid (uuid)
           worker (worker/mk-worker conf

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/clj/backtype/storm/local_state.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/local_state.clj b/storm-core/src/clj/backtype/storm/local_state.clj
index 745d3b0..a8f3334 100644
--- a/storm-core/src/clj/backtype/storm/local_state.clj
+++ b/storm-core/src/clj/backtype/storm/local_state.clj
@@ -69,19 +69,16 @@
                                     (.set_cpu (last resources)))))
     assignment))
 
-(defn mk-local-assignment-with-resources
+(defn mk-local-assignment
   [storm-id executors resources]
   {:storm-id storm-id :executors executors :resources resources})
 
-(defn mk-local-assignment
-  [storm-id executors]
-  {:storm-id storm-id :executors executors})
-
 (defn ->local-assignment
   [^LocalAssignment thrift-local-assignment]
     (mk-local-assignment
       (.get_topology_id thrift-local-assignment)
-      (->executor-list (.get_executors thrift-local-assignment))))
+      (->executor-list (.get_executors thrift-local-assignment))
+      (.get_resources thrift-local-assignment)))
 
 (defn ls-local-assignments!
   [^LocalState local-state assignments]

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/clj/backtype/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index e1a928c..ca43195 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -316,7 +316,7 @@
     (submit-local-topology nimbus storm-name conf topology)))
 
 (defn mk-capture-launch-fn [capture-atom]
-  (fn [supervisor storm-id port worker-id]
+  (fn [supervisor storm-id port worker-id mem-onheap]
     (let [supervisor-id (:supervisor-id supervisor)
           conf (:conf supervisor)
           existing (get @capture-atom [supervisor-id port] [])]

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index a521b10..59dcc53 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1032,17 +1032,25 @@ public class Config extends HashMap<String, Object> {
     public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
 
     /**
-     * The jvm opts provided to workers launched by this supervisor. All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%"
-     * and "%WORKER-PORT%" substrings are replaced with:
+     * 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
+      */
+    public static final String WORKER_HEAP_MEMORY_MB = "worker.heap.memory.mb";
+    public static final Object WORKER_HEAP_MEMORY_MB_SCHEMA = ConfigValidation.PositiveIntegerValidator;
+
+    /**
      * 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.

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/AccessControl.java b/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
deleted file mode 100644
index e0ff680..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
+++ /dev/null
@@ -1,627 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 Integer.valueOf(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 = backtype.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 = backtype.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/a77d0581/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java b/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java
deleted file mode 100644
index e4737ed..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java
+++ /dev/null
@@ -1,406 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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/a77d0581/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java b/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java
deleted file mode 100644
index b803b44..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java
+++ /dev/null
@@ -1,406 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/HBMessage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBMessage.java b/storm-core/src/jvm/backtype/storm/generated/HBMessage.java
deleted file mode 100644
index 8ac963c..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/HBMessage.java
+++ /dev/null
@@ -1,636 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 Integer.valueOf(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 = backtype.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 = backtype.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);
-      }
-    }
-  }
-
-}
-


[08/18] storm git commit: Add resource setting API in WorkerSlot

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/HBMessage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBMessage.java b/storm-core/src/jvm/backtype/storm/generated/HBMessage.java
new file mode 100644
index 0000000..8ac963c
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 Integer.valueOf(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 = backtype.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 = backtype.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/17d70d3b/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBNodes.java b/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
new file mode 100644
index 0000000..7a0b3c1
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 _list694 = iprot.readListBegin();
+                struct.pulseIds = new ArrayList<String>(_list694.size);
+                String _elem695;
+                for (int _i696 = 0; _i696 < _list694.size; ++_i696)
+                {
+                  _elem695 = iprot.readString();
+                  struct.pulseIds.add(_elem695);
+                }
+                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 _iter697 : struct.pulseIds)
+          {
+            oprot.writeString(_iter697);
+          }
+          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 _iter698 : struct.pulseIds)
+          {
+            oprot.writeString(_iter698);
+          }
+        }
+      }
+    }
+
+    @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 _list699 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.pulseIds = new ArrayList<String>(_list699.size);
+          String _elem700;
+          for (int _i701 = 0; _i701 < _list699.size; ++_i701)
+          {
+            _elem700 = iprot.readString();
+            struct.pulseIds.add(_elem700);
+          }
+        }
+        struct.set_pulseIds_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/HBPulse.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBPulse.java b/storm-core/src/jvm/backtype/storm/generated/HBPulse.java
new file mode 100644
index 0000000..0fe0727
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBRecords.java b/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
new file mode 100644
index 0000000..07b923d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 _list686 = iprot.readListBegin();
+                struct.pulses = new ArrayList<HBPulse>(_list686.size);
+                HBPulse _elem687;
+                for (int _i688 = 0; _i688 < _list686.size; ++_i688)
+                {
+                  _elem687 = new HBPulse();
+                  _elem687.read(iprot);
+                  struct.pulses.add(_elem687);
+                }
+                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 _iter689 : struct.pulses)
+          {
+            _iter689.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 _iter690 : struct.pulses)
+          {
+            _iter690.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 _list691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.pulses = new ArrayList<HBPulse>(_list691.size);
+          HBPulse _elem692;
+          for (int _i693 = 0; _i693 < _list691.size; ++_i693)
+          {
+            _elem692 = new HBPulse();
+            _elem692.read(iprot);
+            struct.pulses.add(_elem692);
+          }
+        }
+        struct.set_pulses_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java b/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
new file mode 100644
index 0000000..a5c217e
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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);
+    }
+  }
+
+}
+


[06/18] storm git commit: Add resource setting API in WorkerSlot

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java b/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
new file mode 100644
index 0000000..4898612
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 Long.valueOf(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/17d70d3b/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java b/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
new file mode 100644
index 0000000..2c7992d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 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
+
+  /** 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");
+
+    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;
+        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.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))));
+    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) {
+    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;
+    }
+  }
+
+  public SettableBlobMeta deepCopy() {
+    return new SettableBlobMeta(this);
+  }
+
+  @Override
+  public void clear() {
+    this.acl = null;
+  }
+
+  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 void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACL:
+      if (value == null) {
+        unset_acl();
+      } else {
+        set_acl((List<AccessControl>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACL:
+      return get_acl();
+
+    }
+    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();
+    }
+    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;
+    }
+
+    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);
+
+    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;
+      }
+    }
+    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;
+    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 {
+      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 _list462 = iprot.readListBegin();
+                struct.acl = new ArrayList<AccessControl>(_list462.size);
+                AccessControl _elem463;
+                for (int _i464 = 0; _i464 < _list462.size; ++_i464)
+                {
+                  _elem463 = new AccessControl();
+                  _elem463.read(iprot);
+                  struct.acl.add(_elem463);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_acl_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 _iter465 : struct.acl)
+          {
+            _iter465.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        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 _iter466 : struct.acl)
+        {
+          _iter466.write(oprot);
+        }
+      }
+    }
+
+    @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 _list467 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.acl = new ArrayList<AccessControl>(_list467.size);
+        AccessControl _elem468;
+        for (int _i469 = 0; _i469 < _list467.size; ++_i469)
+        {
+          _elem468 = new AccessControl();
+          _elem468.read(iprot);
+          struct.acl.add(_elem468);
+        }
+      }
+      struct.set_acl_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
new file mode 100644
index 0000000..ab48272
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 _list488 = iprot.readListBegin();
+                struct.topo_ids = new ArrayList<String>(_list488.size);
+                String _elem489;
+                for (int _i490 = 0; _i490 < _list488.size; ++_i490)
+                {
+                  _elem489 = iprot.readString();
+                  struct.topo_ids.add(_elem489);
+                }
+                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 _iter491 : struct.topo_ids)
+          {
+            oprot.writeString(_iter491);
+          }
+          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 _iter492 : struct.topo_ids)
+          {
+            oprot.writeString(_iter492);
+          }
+        }
+      }
+    }
+
+    @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 _list493 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.topo_ids = new ArrayList<String>(_list493.size);
+          String _elem494;
+          for (int _i495 = 0; _i495 < _list493.size; ++_i495)
+          {
+            _elem494 = iprot.readString();
+            struct.topo_ids.add(_elem494);
+          }
+        }
+        struct.set_topo_ids_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java b/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
new file mode 100644
index 0000000..0d4a4c7
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 Double.valueOf(get_mem_on_heap());
+
+    case MEM_OFF_HEAP:
+      return Double.valueOf(get_mem_off_heap());
+
+    case CPU:
+      return Double.valueOf(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);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java b/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
index c89b3bc..b8e4420 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
@@ -20,6 +20,12 @@ package backtype.storm.scheduler;
 public class WorkerSlot {
     String nodeId;
     int port;
+    // amount of on-heap memory allocated to it
+    double memOnHeap = 0.0;
+    // amount of off-heap memory allocated to it
+    double memOffHeap = 0.0;
+    // amount of cpu allocated to it
+    double cpu = 0.0;
     
     public WorkerSlot(String nodeId, Number port) {
         this.nodeId = nodeId;
@@ -34,6 +40,24 @@ public class WorkerSlot {
         return port;
     }
 
+    public void allocateResource(double memOnHeap, double memOffHeap, double cpu) {
+        this.memOnHeap += memOnHeap;
+        this.memOffHeap += memOffHeap;
+        this.cpu += cpu;
+    }
+
+    public double getAllocatedMemOnHeap() {
+        return memOnHeap;
+    }
+
+    public double getAllocatedMemOffHeap() {
+        return memOffHeap;
+    }
+
+    public double getAllocatedCpu() {
+        return cpu;
+    }
+
     @Override
     public int hashCode() {
         return nodeId.hashCode() + 13 * ((Integer) port).hashCode();

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 1b507df..aae851e 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -365,11 +365,17 @@ struct NodeInfo {
     2: required set<i64> port;
 }
 
+struct WorkerResources {
+    1: optional double mem_on_heap;
+    2: optional double mem_off_heap;
+    3: optional double cpu;
+}
 struct Assignment {
     1: required string master_code_dir;
     2: optional map<string, string> node_host = {};
     3: optional map<list<i64>, NodeInfo> executor_node_port = {};
     4: optional map<list<i64>, i64> executor_start_time_secs = {};
+    5: optional map<NodeInfo, WorkerResources> worker_resources = {};
 }
 
 enum TopologyStatus {
@@ -415,6 +421,7 @@ struct LocalStateData {
 struct LocalAssignment {
   1: required string topology_id;
   2: required list<ExecutorInfo> executors;
+  3: optional WorkerResources resources;
 }
 
 struct LSSupervisorId {

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/cluster_test.clj b/storm-core/test/clj/backtype/storm/cluster_test.clj
index c99ee49..198222e 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -170,8 +170,8 @@
 (deftest test-storm-cluster-state-basics
   (with-inprocess-zookeeper zk-port
     (let [state (mk-storm-state zk-port)
-          assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {})
-          assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {})
+          assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {} {})
+          assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {} {})
           nimbusInfo1 (NimbusInfo. "nimbus1" 6667 false)
           nimbusInfo2 (NimbusInfo. "nimbus2" 6667 false)
           nimbusSummary1 (NimbusSummary. "nimbus1" 6667 (current-time-secs) false "v1")

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index bc54d6f..93346d7 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -85,23 +85,27 @@
                        {}))
       (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
       (bind changed (capture-changed-workers
-                        (submit-mocked-assignment
-                          (:nimbus cluster)
-                          "test"
-                          {TOPOLOGY-WORKERS 3}
-                          topology
-                          {1 "1"
-                           2 "1"
-                           3 "1"
-                           4 "1"}
-                          {[1 1] ["sup1" 1]
-                           [2 2] ["sup1" 2]
-                           [3 3] ["sup1" 3]
-                           [4 4] ["sup1" 3]
-                           })
-                        (advance-cluster-time cluster 2)
-                        (heartbeat-workers cluster "sup1" [1 2 3])
-                        (advance-cluster-time cluster 10)))
+                      (submit-mocked-assignment
+                        (:nimbus cluster)
+                        (:storm-cluster-state cluster)
+                        "test"
+                        {TOPOLOGY-WORKERS 3}
+                        topology
+                        {1 "1"
+                         2 "1"
+                         3 "1"
+                         4 "1"}
+                        {[1 1] ["sup1" 1]
+                         [2 2] ["sup1" 2]
+                         [3 3] ["sup1" 3]
+                         [4 4] ["sup1" 3]}
+                        {["sup1" 1] [0.0 0.0 0.0]
+                         ["sup1" 2] [0.0 0.0 0.0]
+                         ["sup1" 3] [0.0 0.0 0.0]
+                         })
+                      (advance-cluster-time cluster 2)
+                      (heartbeat-workers cluster "sup1" [1 2 3])
+                      (advance-cluster-time cluster 10)))
       (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test"))
       (is (empty? (:shutdown changed)))
       (validate-launched-once (:launched changed) {"sup1" [1 2 3]} storm-id)
@@ -136,44 +140,51 @@
       (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
       (bind sup2 (add-supervisor cluster :id "sup2" :ports [1 2]))
       (bind changed (capture-changed-workers
-                        (submit-mocked-assignment
-                          (:nimbus cluster)
-                          "test"
-                          {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 40}
-                          topology
-                          {1 "1"
-                           2 "1"
-                           3 "1"
-                           4 "1"}
-                          {[1 1] ["sup1" 1]
-                           [2 2] ["sup1" 2]
-                           [3 3] ["sup2" 1]
-                           [4 4] ["sup2" 1]
-                           })
-                        (advance-cluster-time cluster 2)
-                        (heartbeat-workers cluster "sup1" [1 2])
-                        (heartbeat-workers cluster "sup2" [1])
-                        ))
+                      (submit-mocked-assignment
+                        (:nimbus cluster)
+                        (:storm-cluster-state cluster)
+                        "test"
+                        {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 40}
+                        topology
+                        {1 "1"
+                         2 "1"
+                         3 "1"
+                         4 "1"}
+                        {[1 1] ["sup1" 1]
+                         [2 2] ["sup1" 2]
+                         [3 3] ["sup2" 1]
+                         [4 4] ["sup2" 1]}
+                        {["sup1" 1] [0.0 0.0 0.0]
+                         ["sup1" 2] [0.0 0.0 0.0]
+                         ["sup2" 1] [0.0 0.0 0.0]
+                         })
+                      (advance-cluster-time cluster 2)
+                      (heartbeat-workers cluster "sup1" [1 2])
+                      (heartbeat-workers cluster "sup2" [1])
+                      ))
       (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test"))
       (is (empty? (:shutdown changed)))
       (validate-launched-once (:launched changed) {"sup1" [1 2] "sup2" [1]} storm-id)
       (bind changed (capture-changed-workers
-                        (submit-mocked-assignment
-                          (:nimbus cluster)
-                          "test2"
-                          {TOPOLOGY-WORKERS 2}
-                          topology2
-                          {1 "1"
-                           2 "1"
-                           3 "1"}
-                          {[1 1] ["sup1" 3]
-                           [2 2] ["sup1" 3]
-                           [3 3] ["sup2" 2]
-                           })
-                        (advance-cluster-time cluster 2)
-                        (heartbeat-workers cluster "sup1" [3])
-                        (heartbeat-workers cluster "sup2" [2])
-                        ))
+                      (submit-mocked-assignment
+                        (:nimbus cluster)
+                        (:storm-cluster-state cluster)
+                        "test2"
+                        {TOPOLOGY-WORKERS 2}
+                        topology2
+                        {1 "1"
+                         2 "1"
+                         3 "1"}
+                        {[1 1] ["sup1" 3]
+                         [2 2] ["sup1" 3]
+                         [3 3] ["sup2" 2]}
+                        {["sup1" 3] [0.0 0.0 0.0]
+                         ["sup2" 2] [0.0 0.0 0.0]
+                         })
+                      (advance-cluster-time cluster 2)
+                      (heartbeat-workers cluster "sup1" [3])
+                      (heartbeat-workers cluster "sup2" [2])
+                      ))
       (bind storm-id2 (get-storm-id (:storm-cluster-state cluster) "test2"))
       (is (empty? (:shutdown changed)))
       (validate-launched-once (:launched changed) {"sup1" [3] "sup2" [2]} storm-id2)


[12/18] storm git commit: Remove generated files

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBNodes.java b/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
deleted file mode 100644
index 7a0b3c1..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
+++ /dev/null
@@ -1,461 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 _list694 = iprot.readListBegin();
-                struct.pulseIds = new ArrayList<String>(_list694.size);
-                String _elem695;
-                for (int _i696 = 0; _i696 < _list694.size; ++_i696)
-                {
-                  _elem695 = iprot.readString();
-                  struct.pulseIds.add(_elem695);
-                }
-                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 _iter697 : struct.pulseIds)
-          {
-            oprot.writeString(_iter697);
-          }
-          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 _iter698 : struct.pulseIds)
-          {
-            oprot.writeString(_iter698);
-          }
-        }
-      }
-    }
-
-    @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 _list699 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.pulseIds = new ArrayList<String>(_list699.size);
-          String _elem700;
-          for (int _i701 = 0; _i701 < _list699.size; ++_i701)
-          {
-            _elem700 = iprot.readString();
-            struct.pulseIds.add(_elem700);
-          }
-        }
-        struct.set_pulseIds_isSet(true);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/HBPulse.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBPulse.java b/storm-core/src/jvm/backtype/storm/generated/HBPulse.java
deleted file mode 100644
index 0fe0727..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/HBPulse.java
+++ /dev/null
@@ -1,522 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBRecords.java b/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
deleted file mode 100644
index 07b923d..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
+++ /dev/null
@@ -1,466 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 _list686 = iprot.readListBegin();
-                struct.pulses = new ArrayList<HBPulse>(_list686.size);
-                HBPulse _elem687;
-                for (int _i688 = 0; _i688 < _list686.size; ++_i688)
-                {
-                  _elem687 = new HBPulse();
-                  _elem687.read(iprot);
-                  struct.pulses.add(_elem687);
-                }
-                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 _iter689 : struct.pulses)
-          {
-            _iter689.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 _iter690 : struct.pulses)
-          {
-            _iter690.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 _list691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.pulses = new ArrayList<HBPulse>(_list691.size);
-          HBPulse _elem692;
-          for (int _i693 = 0; _i693 < _list691.size; ++_i693)
-          {
-            _elem692 = new HBPulse();
-            _elem692.read(iprot);
-            struct.pulses.add(_elem692);
-          }
-        }
-        struct.set_pulses_isSet(true);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java b/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
deleted file mode 100644
index a5c217e..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
+++ /dev/null
@@ -1,406 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java b/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
deleted file mode 100644
index f65633a..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
+++ /dev/null
@@ -1,406 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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);
-    }
-  }
-
-}
-


[11/18] storm git commit: Remove generated files

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
deleted file mode 100644
index 9cda733..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
+++ /dev/null
@@ -1,805 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 Long.valueOf(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 _list662 = iprot.readListBegin();
-                struct.users = new ArrayList<String>(_list662.size);
-                String _elem663;
-                for (int _i664 = 0; _i664 < _list662.size; ++_i664)
-                {
-                  _elem663 = iprot.readString();
-                  struct.users.add(_elem663);
-                }
-                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 _list665 = iprot.readListBegin();
-                struct.groups = new ArrayList<String>(_list665.size);
-                String _elem666;
-                for (int _i667 = 0; _i667 < _list665.size; ++_i667)
-                {
-                  _elem666 = iprot.readString();
-                  struct.groups.add(_elem666);
-                }
-                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 _iter668 : struct.users)
-          {
-            oprot.writeString(_iter668);
-          }
-          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 _iter669 : struct.groups)
-          {
-            oprot.writeString(_iter669);
-          }
-          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 _iter670 : struct.users)
-        {
-          oprot.writeString(_iter670);
-        }
-      }
-      {
-        oprot.writeI32(struct.groups.size());
-        for (String _iter671 : struct.groups)
-        {
-          oprot.writeString(_iter671);
-        }
-      }
-    }
-
-    @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 _list672 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.users = new ArrayList<String>(_list672.size);
-        String _elem673;
-        for (int _i674 = 0; _i674 < _list672.size; ++_i674)
-        {
-          _elem673 = iprot.readString();
-          struct.users.add(_elem673);
-        }
-      }
-      struct.set_users_isSet(true);
-      {
-        org.apache.thrift.protocol.TList _list675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.groups = new ArrayList<String>(_list675.size);
-        String _elem676;
-        for (int _i677 = 0; _i677 < _list675.size; ++_i677)
-        {
-          _elem676 = iprot.readString();
-          struct.groups.add(_elem676);
-        }
-      }
-      struct.set_groups_isSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
deleted file mode 100644
index b0fc6f4..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
+++ /dev/null
@@ -1,460 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 _list678 = iprot.readListBegin();
-                struct.topo_history = new ArrayList<LSTopoHistory>(_list678.size);
-                LSTopoHistory _elem679;
-                for (int _i680 = 0; _i680 < _list678.size; ++_i680)
-                {
-                  _elem679 = new LSTopoHistory();
-                  _elem679.read(iprot);
-                  struct.topo_history.add(_elem679);
-                }
-                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 _iter681 : struct.topo_history)
-          {
-            _iter681.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 _iter682 : struct.topo_history)
-        {
-          _iter682.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 _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.topo_history = new ArrayList<LSTopoHistory>(_list683.size);
-        LSTopoHistory _elem684;
-        for (int _i685 = 0; _i685 < _list683.size; ++_i685)
-        {
-          _elem684 = new LSTopoHistory();
-          _elem684.read(iprot);
-          struct.topo_history.add(_elem684);
-        }
-      }
-      struct.set_topo_history_isSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java b/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
deleted file mode 100644
index 98b412c..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
+++ /dev/null
@@ -1,556 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 _list470 = iprot.readListBegin();
-                struct.keys = new ArrayList<String>(_list470.size);
-                String _elem471;
-                for (int _i472 = 0; _i472 < _list470.size; ++_i472)
-                {
-                  _elem471 = iprot.readString();
-                  struct.keys.add(_elem471);
-                }
-                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 _iter473 : struct.keys)
-          {
-            oprot.writeString(_iter473);
-          }
-          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 _iter474 : struct.keys)
-        {
-          oprot.writeString(_iter474);
-        }
-      }
-      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 _list475 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.keys = new ArrayList<String>(_list475.size);
-        String _elem476;
-        for (int _i477 = 0; _i477 < _list475.size; ++_i477)
-        {
-          _elem476 = iprot.readString();
-          struct.keys.add(_elem476);
-        }
-      }
-      struct.set_keys_isSet(true);
-      struct.session = iprot.readString();
-      struct.set_session_isSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java b/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
deleted file mode 100644
index 4898612..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
+++ /dev/null
@@ -1,510 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 Long.valueOf(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);
-    }
-  }
-
-}
-


[07/18] storm git commit: Add resource setting API in WorkerSlot

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java b/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
new file mode 100644
index 0000000..f65633a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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/17d70d3b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
new file mode 100644
index 0000000..9cda733
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 Long.valueOf(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 _list662 = iprot.readListBegin();
+                struct.users = new ArrayList<String>(_list662.size);
+                String _elem663;
+                for (int _i664 = 0; _i664 < _list662.size; ++_i664)
+                {
+                  _elem663 = iprot.readString();
+                  struct.users.add(_elem663);
+                }
+                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 _list665 = iprot.readListBegin();
+                struct.groups = new ArrayList<String>(_list665.size);
+                String _elem666;
+                for (int _i667 = 0; _i667 < _list665.size; ++_i667)
+                {
+                  _elem666 = iprot.readString();
+                  struct.groups.add(_elem666);
+                }
+                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 _iter668 : struct.users)
+          {
+            oprot.writeString(_iter668);
+          }
+          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 _iter669 : struct.groups)
+          {
+            oprot.writeString(_iter669);
+          }
+          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 _iter670 : struct.users)
+        {
+          oprot.writeString(_iter670);
+        }
+      }
+      {
+        oprot.writeI32(struct.groups.size());
+        for (String _iter671 : struct.groups)
+        {
+          oprot.writeString(_iter671);
+        }
+      }
+    }
+
+    @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 _list672 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.users = new ArrayList<String>(_list672.size);
+        String _elem673;
+        for (int _i674 = 0; _i674 < _list672.size; ++_i674)
+        {
+          _elem673 = iprot.readString();
+          struct.users.add(_elem673);
+        }
+      }
+      struct.set_users_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.groups = new ArrayList<String>(_list675.size);
+        String _elem676;
+        for (int _i677 = 0; _i677 < _list675.size; ++_i677)
+        {
+          _elem676 = iprot.readString();
+          struct.groups.add(_elem676);
+        }
+      }
+      struct.set_groups_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
new file mode 100644
index 0000000..b0fc6f4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 _list678 = iprot.readListBegin();
+                struct.topo_history = new ArrayList<LSTopoHistory>(_list678.size);
+                LSTopoHistory _elem679;
+                for (int _i680 = 0; _i680 < _list678.size; ++_i680)
+                {
+                  _elem679 = new LSTopoHistory();
+                  _elem679.read(iprot);
+                  struct.topo_history.add(_elem679);
+                }
+                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 _iter681 : struct.topo_history)
+          {
+            _iter681.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 _iter682 : struct.topo_history)
+        {
+          _iter682.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 _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topo_history = new ArrayList<LSTopoHistory>(_list683.size);
+        LSTopoHistory _elem684;
+        for (int _i685 = 0; _i685 < _list683.size; ++_i685)
+        {
+          _elem684 = new LSTopoHistory();
+          _elem684.read(iprot);
+          struct.topo_history.add(_elem684);
+        }
+      }
+      struct.set_topo_history_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java b/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
new file mode 100644
index 0000000..98b412c
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 _list470 = iprot.readListBegin();
+                struct.keys = new ArrayList<String>(_list470.size);
+                String _elem471;
+                for (int _i472 = 0; _i472 < _list470.size; ++_i472)
+                {
+                  _elem471 = iprot.readString();
+                  struct.keys.add(_elem471);
+                }
+                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 _iter473 : struct.keys)
+          {
+            oprot.writeString(_iter473);
+          }
+          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 _iter474 : struct.keys)
+        {
+          oprot.writeString(_iter474);
+        }
+      }
+      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 _list475 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.keys = new ArrayList<String>(_list475.size);
+        String _elem476;
+        for (int _i477 = 0; _i477 < _list475.size; ++_i477)
+        {
+          _elem476 = iprot.readString();
+          struct.keys.add(_elem476);
+        }
+      }
+      struct.set_keys_isSet(true);
+      struct.session = iprot.readString();
+      struct.set_session_isSet(true);
+    }
+  }
+
+}
+


[02/18] storm git commit: Push Thrift file changes

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/584ded8e/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index 4df4521..f91442f 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -6510,6 +6510,97 @@ class NodeInfo:
   def __ne__(self, other):
     return not (self == other)
 
+class WorkerResources:
+  """
+  Attributes:
+   - mem_on_heap
+   - mem_off_heap
+   - cpu
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.DOUBLE, 'mem_on_heap', None, None, ), # 1
+    (2, TType.DOUBLE, 'mem_off_heap', None, None, ), # 2
+    (3, TType.DOUBLE, 'cpu', None, None, ), # 3
+  )
+
+  def __init__(self, mem_on_heap=None, mem_off_heap=None, cpu=None,):
+    self.mem_on_heap = mem_on_heap
+    self.mem_off_heap = mem_off_heap
+    self.cpu = cpu
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.DOUBLE:
+          self.mem_on_heap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.DOUBLE:
+          self.mem_off_heap = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.DOUBLE:
+          self.cpu = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WorkerResources')
+    if self.mem_on_heap is not None:
+      oprot.writeFieldBegin('mem_on_heap', TType.DOUBLE, 1)
+      oprot.writeDouble(self.mem_on_heap)
+      oprot.writeFieldEnd()
+    if self.mem_off_heap is not None:
+      oprot.writeFieldBegin('mem_off_heap', TType.DOUBLE, 2)
+      oprot.writeDouble(self.mem_off_heap)
+      oprot.writeFieldEnd()
+    if self.cpu is not None:
+      oprot.writeFieldBegin('cpu', TType.DOUBLE, 3)
+      oprot.writeDouble(self.cpu)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.mem_on_heap)
+    value = (value * 31) ^ hash(self.mem_off_heap)
+    value = (value * 31) ^ hash(self.cpu)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class Assignment:
   """
   Attributes:
@@ -6517,6 +6608,7 @@ class Assignment:
    - node_host
    - executor_node_port
    - executor_start_time_secs
+   - worker_resources
   """
 
   thrift_spec = (
@@ -6528,9 +6620,11 @@ class Assignment:
     }, ), # 3
     (4, TType.MAP, 'executor_start_time_secs', (TType.LIST,(TType.I64,None),TType.I64,None), {
     }, ), # 4
+    (5, TType.MAP, 'worker_resources', (TType.STRUCT,(NodeInfo, NodeInfo.thrift_spec),TType.STRUCT,(WorkerResources, WorkerResources.thrift_spec)), {
+    }, ), # 5
   )
 
-  def __init__(self, master_code_dir=None, node_host=thrift_spec[2][4], executor_node_port=thrift_spec[3][4], executor_start_time_secs=thrift_spec[4][4],):
+  def __init__(self, master_code_dir=None, node_host=thrift_spec[2][4], executor_node_port=thrift_spec[3][4], executor_start_time_secs=thrift_spec[4][4], worker_resources=thrift_spec[5][4],):
     self.master_code_dir = master_code_dir
     if node_host is self.thrift_spec[2][4]:
       node_host = {
@@ -6544,6 +6638,10 @@ class Assignment:
       executor_start_time_secs = {
     }
     self.executor_start_time_secs = executor_start_time_secs
+    if worker_resources is self.thrift_spec[5][4]:
+      worker_resources = {
+    }
+    self.worker_resources = worker_resources
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -6603,6 +6701,19 @@ class Assignment:
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.MAP:
+          self.worker_resources = {}
+          (_ktype503, _vtype504, _size502 ) = iprot.readMapBegin()
+          for _i506 in xrange(_size502):
+            _key507 = NodeInfo()
+            _key507.read(iprot)
+            _val508 = WorkerResources()
+            _val508.read(iprot)
+            self.worker_resources[_key507] = _val508
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -6620,31 +6731,39 @@ class Assignment:
     if self.node_host is not None:
       oprot.writeFieldBegin('node_host', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.node_host))
-      for kiter502,viter503 in self.node_host.items():
-        oprot.writeString(kiter502.encode('utf-8'))
-        oprot.writeString(viter503.encode('utf-8'))
+      for kiter509,viter510 in self.node_host.items():
+        oprot.writeString(kiter509.encode('utf-8'))
+        oprot.writeString(viter510.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_node_port is not None:
       oprot.writeFieldBegin('executor_node_port', TType.MAP, 3)
       oprot.writeMapBegin(TType.LIST, TType.STRUCT, len(self.executor_node_port))
-      for kiter504,viter505 in self.executor_node_port.items():
-        oprot.writeListBegin(TType.I64, len(kiter504))
-        for iter506 in kiter504:
-          oprot.writeI64(iter506)
+      for kiter511,viter512 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter511))
+        for iter513 in kiter511:
+          oprot.writeI64(iter513)
         oprot.writeListEnd()
-        viter505.write(oprot)
+        viter512.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_start_time_secs is not None:
       oprot.writeFieldBegin('executor_start_time_secs', TType.MAP, 4)
       oprot.writeMapBegin(TType.LIST, TType.I64, len(self.executor_start_time_secs))
-      for kiter507,viter508 in self.executor_start_time_secs.items():
-        oprot.writeListBegin(TType.I64, len(kiter507))
-        for iter509 in kiter507:
-          oprot.writeI64(iter509)
+      for kiter514,viter515 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter514))
+        for iter516 in kiter514:
+          oprot.writeI64(iter516)
         oprot.writeListEnd()
-        oprot.writeI64(viter508)
+        oprot.writeI64(viter515)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.worker_resources is not None:
+      oprot.writeFieldBegin('worker_resources', TType.MAP, 5)
+      oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.worker_resources))
+      for kiter517,viter518 in self.worker_resources.items():
+        kiter517.write(oprot)
+        viter518.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6662,6 +6781,7 @@ class Assignment:
     value = (value * 31) ^ hash(self.node_host)
     value = (value * 31) ^ hash(self.executor_node_port)
     value = (value * 31) ^ hash(self.executor_start_time_secs)
+    value = (value * 31) ^ hash(self.worker_resources)
     return value
 
   def __repr__(self):
@@ -6820,11 +6940,11 @@ class StormBase:
       elif fid == 4:
         if ftype == TType.MAP:
           self.component_executors = {}
-          (_ktype511, _vtype512, _size510 ) = iprot.readMapBegin()
-          for _i514 in xrange(_size510):
-            _key515 = iprot.readString().decode('utf-8')
-            _val516 = iprot.readI32();
-            self.component_executors[_key515] = _val516
+          (_ktype520, _vtype521, _size519 ) = iprot.readMapBegin()
+          for _i523 in xrange(_size519):
+            _key524 = iprot.readString().decode('utf-8')
+            _val525 = iprot.readI32();
+            self.component_executors[_key524] = _val525
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6852,12 +6972,12 @@ class StormBase:
       elif fid == 9:
         if ftype == TType.MAP:
           self.component_debug = {}
-          (_ktype518, _vtype519, _size517 ) = iprot.readMapBegin()
-          for _i521 in xrange(_size517):
-            _key522 = iprot.readString().decode('utf-8')
-            _val523 = DebugOptions()
-            _val523.read(iprot)
-            self.component_debug[_key522] = _val523
+          (_ktype527, _vtype528, _size526 ) = iprot.readMapBegin()
+          for _i530 in xrange(_size526):
+            _key531 = iprot.readString().decode('utf-8')
+            _val532 = DebugOptions()
+            _val532.read(iprot)
+            self.component_debug[_key531] = _val532
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6886,9 +7006,9 @@ class StormBase:
     if self.component_executors is not None:
       oprot.writeFieldBegin('component_executors', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.component_executors))
-      for kiter524,viter525 in self.component_executors.items():
-        oprot.writeString(kiter524.encode('utf-8'))
-        oprot.writeI32(viter525)
+      for kiter533,viter534 in self.component_executors.items():
+        oprot.writeString(kiter533.encode('utf-8'))
+        oprot.writeI32(viter534)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.launch_time_secs is not None:
@@ -6910,9 +7030,9 @@ class StormBase:
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
-      for kiter526,viter527 in self.component_debug.items():
-        oprot.writeString(kiter526.encode('utf-8'))
-        viter527.write(oprot)
+      for kiter535,viter536 in self.component_debug.items():
+        oprot.writeString(kiter535.encode('utf-8'))
+        viter536.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6992,13 +7112,13 @@ class ClusterWorkerHeartbeat:
       elif fid == 2:
         if ftype == TType.MAP:
           self.executor_stats = {}
-          (_ktype529, _vtype530, _size528 ) = iprot.readMapBegin()
-          for _i532 in xrange(_size528):
-            _key533 = ExecutorInfo()
-            _key533.read(iprot)
-            _val534 = ExecutorStats()
-            _val534.read(iprot)
-            self.executor_stats[_key533] = _val534
+          (_ktype538, _vtype539, _size537 ) = iprot.readMapBegin()
+          for _i541 in xrange(_size537):
+            _key542 = ExecutorInfo()
+            _key542.read(iprot)
+            _val543 = ExecutorStats()
+            _val543.read(iprot)
+            self.executor_stats[_key542] = _val543
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7029,9 +7149,9 @@ class ClusterWorkerHeartbeat:
     if self.executor_stats is not None:
       oprot.writeFieldBegin('executor_stats', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.executor_stats))
-      for kiter535,viter536 in self.executor_stats.items():
-        kiter535.write(oprot)
-        viter536.write(oprot)
+      for kiter544,viter545 in self.executor_stats.items():
+        kiter544.write(oprot)
+        viter545.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.time_secs is not None:
@@ -7184,12 +7304,12 @@ class LocalStateData:
       if fid == 1:
         if ftype == TType.MAP:
           self.serialized_parts = {}
-          (_ktype538, _vtype539, _size537 ) = iprot.readMapBegin()
-          for _i541 in xrange(_size537):
-            _key542 = iprot.readString().decode('utf-8')
-            _val543 = ThriftSerializedObject()
-            _val543.read(iprot)
-            self.serialized_parts[_key542] = _val543
+          (_ktype547, _vtype548, _size546 ) = iprot.readMapBegin()
+          for _i550 in xrange(_size546):
+            _key551 = iprot.readString().decode('utf-8')
+            _val552 = ThriftSerializedObject()
+            _val552.read(iprot)
+            self.serialized_parts[_key551] = _val552
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7206,9 +7326,9 @@ class LocalStateData:
     if self.serialized_parts is not None:
       oprot.writeFieldBegin('serialized_parts', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.serialized_parts))
-      for kiter544,viter545 in self.serialized_parts.items():
-        oprot.writeString(kiter544.encode('utf-8'))
-        viter545.write(oprot)
+      for kiter553,viter554 in self.serialized_parts.items():
+        oprot.writeString(kiter553.encode('utf-8'))
+        viter554.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7241,17 +7361,20 @@ class LocalAssignment:
   Attributes:
    - topology_id
    - executors
+   - resources
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRING, 'topology_id', None, None, ), # 1
     (2, TType.LIST, 'executors', (TType.STRUCT,(ExecutorInfo, ExecutorInfo.thrift_spec)), None, ), # 2
+    (3, TType.STRUCT, 'resources', (WorkerResources, WorkerResources.thrift_spec), None, ), # 3
   )
 
-  def __init__(self, topology_id=None, executors=None,):
+  def __init__(self, topology_id=None, executors=None, resources=None,):
     self.topology_id = topology_id
     self.executors = executors
+    self.resources = resources
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -7270,14 +7393,20 @@ class LocalAssignment:
       elif fid == 2:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype549, _size546) = iprot.readListBegin()
-          for _i550 in xrange(_size546):
-            _elem551 = ExecutorInfo()
-            _elem551.read(iprot)
-            self.executors.append(_elem551)
+          (_etype558, _size555) = iprot.readListBegin()
+          for _i559 in xrange(_size555):
+            _elem560 = ExecutorInfo()
+            _elem560.read(iprot)
+            self.executors.append(_elem560)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.resources = WorkerResources()
+          self.resources.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -7295,10 +7424,14 @@ class LocalAssignment:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter552 in self.executors:
-        iter552.write(oprot)
+      for iter561 in self.executors:
+        iter561.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
+    if self.resources is not None:
+      oprot.writeFieldBegin('resources', TType.STRUCT, 3)
+      self.resources.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -7314,6 +7447,7 @@ class LocalAssignment:
     value = 17
     value = (value * 31) ^ hash(self.topology_id)
     value = (value * 31) ^ hash(self.executors)
+    value = (value * 31) ^ hash(self.resources)
     return value
 
   def __repr__(self):
@@ -7420,11 +7554,11 @@ class LSApprovedWorkers:
       if fid == 1:
         if ftype == TType.MAP:
           self.approved_workers = {}
-          (_ktype554, _vtype555, _size553 ) = iprot.readMapBegin()
-          for _i557 in xrange(_size553):
-            _key558 = iprot.readString().decode('utf-8')
-            _val559 = iprot.readI32();
-            self.approved_workers[_key558] = _val559
+          (_ktype563, _vtype564, _size562 ) = iprot.readMapBegin()
+          for _i566 in xrange(_size562):
+            _key567 = iprot.readString().decode('utf-8')
+            _val568 = iprot.readI32();
+            self.approved_workers[_key567] = _val568
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7441,9 +7575,9 @@ class LSApprovedWorkers:
     if self.approved_workers is not None:
       oprot.writeFieldBegin('approved_workers', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.approved_workers))
-      for kiter560,viter561 in self.approved_workers.items():
-        oprot.writeString(kiter560.encode('utf-8'))
-        oprot.writeI32(viter561)
+      for kiter569,viter570 in self.approved_workers.items():
+        oprot.writeString(kiter569.encode('utf-8'))
+        oprot.writeI32(viter570)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7497,12 +7631,12 @@ class LSSupervisorAssignments:
       if fid == 1:
         if ftype == TType.MAP:
           self.assignments = {}
-          (_ktype563, _vtype564, _size562 ) = iprot.readMapBegin()
-          for _i566 in xrange(_size562):
-            _key567 = iprot.readI32();
-            _val568 = LocalAssignment()
-            _val568.read(iprot)
-            self.assignments[_key567] = _val568
+          (_ktype572, _vtype573, _size571 ) = iprot.readMapBegin()
+          for _i575 in xrange(_size571):
+            _key576 = iprot.readI32();
+            _val577 = LocalAssignment()
+            _val577.read(iprot)
+            self.assignments[_key576] = _val577
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7519,9 +7653,9 @@ class LSSupervisorAssignments:
     if self.assignments is not None:
       oprot.writeFieldBegin('assignments', TType.MAP, 1)
       oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments))
-      for kiter569,viter570 in self.assignments.items():
-        oprot.writeI32(kiter569)
-        viter570.write(oprot)
+      for kiter578,viter579 in self.assignments.items():
+        oprot.writeI32(kiter578)
+        viter579.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7594,11 +7728,11 @@ class LSWorkerHeartbeat:
       elif fid == 3:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype574, _size571) = iprot.readListBegin()
-          for _i575 in xrange(_size571):
-            _elem576 = ExecutorInfo()
-            _elem576.read(iprot)
-            self.executors.append(_elem576)
+          (_etype583, _size580) = iprot.readListBegin()
+          for _i584 in xrange(_size580):
+            _elem585 = ExecutorInfo()
+            _elem585.read(iprot)
+            self.executors.append(_elem585)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7628,8 +7762,8 @@ class LSWorkerHeartbeat:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter577 in self.executors:
-        iter577.write(oprot)
+      for iter586 in self.executors:
+        iter586.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.port is not None:
@@ -7881,12 +8015,12 @@ class LogConfig:
       if fid == 2:
         if ftype == TType.MAP:
           self.named_logger_level = {}
-          (_ktype579, _vtype580, _size578 ) = iprot.readMapBegin()
-          for _i582 in xrange(_size578):
-            _key583 = iprot.readString().decode('utf-8')
-            _val584 = LogLevel()
-            _val584.read(iprot)
-            self.named_logger_level[_key583] = _val584
+          (_ktype588, _vtype589, _size587 ) = iprot.readMapBegin()
+          for _i591 in xrange(_size587):
+            _key592 = iprot.readString().decode('utf-8')
+            _val593 = LogLevel()
+            _val593.read(iprot)
+            self.named_logger_level[_key592] = _val593
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7903,9 +8037,9 @@ class LogConfig:
     if self.named_logger_level is not None:
       oprot.writeFieldBegin('named_logger_level', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.named_logger_level))
-      for kiter585,viter586 in self.named_logger_level.items():
-        oprot.writeString(kiter585.encode('utf-8'))
-        viter586.write(oprot)
+      for kiter594,viter595 in self.named_logger_level.items():
+        oprot.writeString(kiter594.encode('utf-8'))
+        viter595.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()


[17/18] storm git commit: Merge branch '1093a' of https://github.com/zhuoliu/storm

Posted by kn...@apache.org.
Merge branch '1093a' of https://github.com/zhuoliu/storm


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

Branch: refs/heads/master
Commit: c88a4f69d8525cbcefb24863e0b893d8226ce650
Parents: 1d393ee b0e52ac
Author: Kyle Nusbaum <Ky...@gmail.com>
Authored: Mon Oct 19 15:05:26 2015 -0500
Committer: Kyle Nusbaum <Ky...@gmail.com>
Committed: Mon Oct 19 15:05:26 2015 -0500

----------------------------------------------------------------------
 conf/defaults.yaml                              |   3 +-
 storm-core/src/clj/backtype/storm/converter.clj |  50 +-
 .../src/clj/backtype/storm/daemon/common.clj    |   2 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  99 +--
 .../clj/backtype/storm/daemon/supervisor.clj    |  49 +-
 .../src/clj/backtype/storm/local_state.clj      |  20 +-
 storm-core/src/clj/backtype/storm/testing.clj   |  31 +-
 storm-core/src/jvm/backtype/storm/Config.java   |  13 +-
 .../storm/generated/AlreadyAliveException.java  |   2 +-
 .../backtype/storm/generated/Assignment.java    | 308 ++++++++--
 .../storm/generated/AuthorizationException.java |   2 +-
 .../src/jvm/backtype/storm/generated/Bolt.java  |   2 +-
 .../storm/generated/BoltAggregateStats.java     |   2 +-
 .../jvm/backtype/storm/generated/BoltStats.java |   2 +-
 .../storm/generated/ClusterSummary.java         |   2 +-
 .../storm/generated/ClusterWorkerHeartbeat.java |  54 +-
 .../storm/generated/CommonAggregateStats.java   |   2 +-
 .../generated/ComponentAggregateStats.java      |   2 +-
 .../storm/generated/ComponentCommon.java        |   2 +-
 .../storm/generated/ComponentPageInfo.java      |   2 +-
 .../backtype/storm/generated/Credentials.java   |   2 +-
 .../storm/generated/DRPCExecutionException.java |   2 +-
 .../backtype/storm/generated/DRPCRequest.java   |   2 +-
 .../backtype/storm/generated/DebugOptions.java  |   2 +-
 .../storm/generated/DistributedRPC.java         |   2 +-
 .../generated/DistributedRPCInvocations.java    |   2 +-
 .../jvm/backtype/storm/generated/ErrorInfo.java |   2 +-
 .../storm/generated/ExecutorAggregateStats.java |   2 +-
 .../backtype/storm/generated/ExecutorInfo.java  |   2 +-
 .../backtype/storm/generated/ExecutorStats.java |   2 +-
 .../storm/generated/ExecutorSummary.java        |   2 +-
 .../storm/generated/GetInfoOptions.java         |   2 +-
 .../storm/generated/GlobalStreamId.java         |   2 +-
 .../generated/InvalidTopologyException.java     |   2 +-
 .../backtype/storm/generated/JavaObject.java    |   2 +-
 .../backtype/storm/generated/KillOptions.java   |   2 +-
 .../storm/generated/LSApprovedWorkers.java      |  46 +-
 .../generated/LSSupervisorAssignments.java      |  50 +-
 .../storm/generated/LSSupervisorId.java         |   2 +-
 .../storm/generated/LSWorkerHeartbeat.java      |  38 +-
 .../storm/generated/LocalAssignment.java        | 155 ++++-
 .../storm/generated/LocalStateData.java         |  50 +-
 .../jvm/backtype/storm/generated/LogConfig.java |  50 +-
 .../jvm/backtype/storm/generated/LogLevel.java  |   2 +-
 .../jvm/backtype/storm/generated/Nimbus.java    |   2 +-
 .../backtype/storm/generated/NimbusSummary.java |   2 +-
 .../jvm/backtype/storm/generated/NodeInfo.java  |   2 +-
 .../storm/generated/NotAliveException.java      |   2 +-
 .../backtype/storm/generated/NullStruct.java    |   2 +-
 .../storm/generated/RebalanceOptions.java       |   2 +-
 .../storm/generated/ShellComponent.java         |   2 +-
 .../storm/generated/SpoutAggregateStats.java    |   2 +-
 .../jvm/backtype/storm/generated/SpoutSpec.java |   2 +-
 .../backtype/storm/generated/SpoutStats.java    |   2 +-
 .../storm/generated/StateSpoutSpec.java         |   2 +-
 .../jvm/backtype/storm/generated/StormBase.java |  94 +--
 .../backtype/storm/generated/StormTopology.java |   2 +-
 .../backtype/storm/generated/StreamInfo.java    |   2 +-
 .../backtype/storm/generated/SubmitOptions.java |   2 +-
 .../storm/generated/SupervisorInfo.java         |   2 +-
 .../storm/generated/SupervisorSummary.java      |   2 +-
 .../storm/generated/ThriftSerializedObject.java |   2 +-
 .../backtype/storm/generated/TopologyInfo.java  |   2 +-
 .../storm/generated/TopologyPageInfo.java       |   2 +-
 .../backtype/storm/generated/TopologyStats.java |   2 +-
 .../storm/generated/TopologySummary.java        |   2 +-
 .../storm/generated/WorkerResources.java        | 605 +++++++++++++++++++
 .../backtype/storm/scheduler/WorkerSlot.java    |  25 +
 .../storm/scheduler/resource/RAS_Node.java      |  28 +
 storm-core/src/py/storm/ttypes.py               | 316 +++++++---
 storm-core/src/storm.thrift                     |   7 +
 .../test/clj/backtype/storm/cluster_test.clj    |   4 +-
 .../test/clj/backtype/storm/supervisor_test.clj | 176 +++---
 73 files changed, 1793 insertions(+), 578 deletions(-)
----------------------------------------------------------------------



[10/18] storm git commit: Remove generated files

Posted by kn...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java b/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
deleted file mode 100644
index 2c7992d..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
+++ /dev/null
@@ -1,460 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 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
-
-  /** 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");
-
-    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;
-        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.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))));
-    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) {
-    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;
-    }
-  }
-
-  public SettableBlobMeta deepCopy() {
-    return new SettableBlobMeta(this);
-  }
-
-  @Override
-  public void clear() {
-    this.acl = null;
-  }
-
-  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 void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case ACL:
-      if (value == null) {
-        unset_acl();
-      } else {
-        set_acl((List<AccessControl>)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case ACL:
-      return get_acl();
-
-    }
-    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();
-    }
-    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;
-    }
-
-    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);
-
-    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;
-      }
-    }
-    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;
-    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 {
-      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 _list462 = iprot.readListBegin();
-                struct.acl = new ArrayList<AccessControl>(_list462.size);
-                AccessControl _elem463;
-                for (int _i464 = 0; _i464 < _list462.size; ++_i464)
-                {
-                  _elem463 = new AccessControl();
-                  _elem463.read(iprot);
-                  struct.acl.add(_elem463);
-                }
-                iprot.readListEnd();
-              }
-              struct.set_acl_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 _iter465 : struct.acl)
-          {
-            _iter465.write(oprot);
-          }
-          oprot.writeListEnd();
-        }
-        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 _iter466 : struct.acl)
-        {
-          _iter466.write(oprot);
-        }
-      }
-    }
-
-    @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 _list467 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.acl = new ArrayList<AccessControl>(_list467.size);
-        AccessControl _elem468;
-        for (int _i469 = 0; _i469 < _list467.size; ++_i469)
-        {
-          _elem468 = new AccessControl();
-          _elem468.read(iprot);
-          struct.acl.add(_elem468);
-        }
-      }
-      struct.set_acl_isSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
deleted file mode 100644
index ab48272..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
+++ /dev/null
@@ -1,461 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 _list488 = iprot.readListBegin();
-                struct.topo_ids = new ArrayList<String>(_list488.size);
-                String _elem489;
-                for (int _i490 = 0; _i490 < _list488.size; ++_i490)
-                {
-                  _elem489 = iprot.readString();
-                  struct.topo_ids.add(_elem489);
-                }
-                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 _iter491 : struct.topo_ids)
-          {
-            oprot.writeString(_iter491);
-          }
-          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 _iter492 : struct.topo_ids)
-          {
-            oprot.writeString(_iter492);
-          }
-        }
-      }
-    }
-
-    @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 _list493 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.topo_ids = new ArrayList<String>(_list493.size);
-          String _elem494;
-          for (int _i495 = 0; _i495 < _list493.size; ++_i495)
-          {
-            _elem494 = iprot.readString();
-            struct.topo_ids.add(_elem494);
-          }
-        }
-        struct.set_topo_ids_isSet(true);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java b/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
deleted file mode 100644
index 0d4a4c7..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
+++ /dev/null
@@ -1,605 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.2)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package backtype.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.2)", date = "2015-8-5")
-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 Double.valueOf(get_mem_on_heap());
-
-    case MEM_OFF_HEAP:
-      return Double.valueOf(get_mem_off_heap());
-
-    case CPU:
-      return Double.valueOf(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);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java b/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
index b8e4420..25892df 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
@@ -40,10 +40,11 @@ public class WorkerSlot {
         return port;
     }
 
-    public void allocateResource(double memOnHeap, double memOffHeap, double cpu) {
+    public WorkerSlot allocateResource(double memOnHeap, double memOffHeap, double cpu) {
         this.memOnHeap += memOnHeap;
         this.memOffHeap += memOffHeap;
         this.cpu += cpu;
+        return this;
     }
 
     public double getAllocatedMemOnHeap() {

http://git-wip-us.apache.org/repos/asf/storm/blob/a77d0581/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index 93346d7..8303754 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -262,6 +262,7 @@
     (let [mock-port "42"
           mock-storm-id "fake-storm-id"
           mock-worker-id "fake-worker-id"
+          mock-mem-onheap 512
           mock-cp (str file-path-separator "base" class-path-separator file-path-separator "stormjar.jar")
           mock-sensitivity "S3"
           mock-cp "/base:/stormjar.jar"
@@ -315,7 +316,8 @@
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
-                                      mock-worker-id)
+                                      mock-worker-id
+                                      mock-mem-onheap)
             (verify-first-call-args-for-indices launch-process
                                                 [0]
                                                 exp-args))))
@@ -336,7 +338,8 @@
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
-                                      mock-worker-id)
+                                      mock-worker-id
+                                      mock-mem-onheap)
             (verify-first-call-args-for-indices launch-process
                                                 [0]
                                                 exp-args))))
@@ -354,7 +357,8 @@
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
-                                              mock-worker-id)
+                                              mock-worker-id
+                                              mock-mem-onheap)
                     (verify-first-call-args-for-indices launch-process
                                                         [0]
                                                         exp-args))))
@@ -373,7 +377,8 @@
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
-                                              mock-worker-id)
+                                              mock-worker-id
+                                              mock-mem-onheap)
                     (verify-first-call-args-for-indices launch-process
                                                         [2]
                                                         full-env)))))))
@@ -389,6 +394,7 @@
     (let [mock-port "42"
           mock-storm-id "fake-storm-id"
           mock-worker-id "fake-worker-id"
+          mock-mem-onheap 512
           mock-sensitivity "S3"
           mock-cp "mock-classpath'quote-on-purpose"
           storm-local (str "/tmp/" (UUID/randomUUID))
@@ -454,7 +460,8 @@
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
-                                      mock-worker-id)
+                                      mock-worker-id
+                                      mock-mem-onheap)
             (verify-first-call-args-for-indices launch-process
                                                 [0]
                                                 exp-launch))
@@ -480,7 +487,8 @@
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
-                                      mock-worker-id)
+                                      mock-worker-id
+                                      mock-mem-onheap)
             (verify-first-call-args-for-indices launch-process
                                                 [0]
                                                 exp-launch))
@@ -569,9 +577,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
-          childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m"
-          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          mem-onheap 512
+          childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
+          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-happy-path-list
@@ -579,9 +588,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
-          childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m")
-          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          mem-onheap 512
+          childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
+          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-happy-path-list-arraylist
@@ -599,9 +609,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-no-keys
@@ -609,9 +620,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-nil-childopts
@@ -619,9 +631,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts nil
           expected-childopts nil
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-nil-ids
@@ -629,9 +642,10 @@
     (let [worker-id nil
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-retry-read-assignments


[09/18] storm git commit: Add resource setting API in WorkerSlot

Posted by kn...@apache.org.
Add resource setting API in WorkerSlot

Merge pull request #509 from zhuol/1779

[YSTORM-1779] Send/receive resource information from nimbus to supervisor for RAS


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

Branch: refs/heads/master
Commit: 17d70d3b0f163e6ca3500d2c96b3e34850271a28
Parents: 1d393ee
Author: zhuol <zh...@yahoo-inc.com>
Authored: Wed Oct 7 16:08:36 2015 -0500
Committer: zhuol <zh...@yahoo-inc.com>
Committed: Mon Oct 19 09:38:44 2015 -0500

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/converter.clj |  50 +-
 .../src/clj/backtype/storm/daemon/common.clj    |   2 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  88 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  20 +-
 .../src/clj/backtype/storm/local_state.clj      |  17 +-
 storm-core/src/clj/backtype/storm/testing.clj   |  29 +-
 .../backtype/storm/generated/AccessControl.java | 627 +++++++++++++++
 .../generated/HBAuthorizationException.java     | 406 ++++++++++
 .../storm/generated/HBExecutionException.java   | 406 ++++++++++
 .../jvm/backtype/storm/generated/HBMessage.java | 636 +++++++++++++++
 .../jvm/backtype/storm/generated/HBNodes.java   | 461 +++++++++++
 .../jvm/backtype/storm/generated/HBPulse.java   | 522 ++++++++++++
 .../jvm/backtype/storm/generated/HBRecords.java | 466 +++++++++++
 .../generated/KeyAlreadyExistsException.java    | 406 ++++++++++
 .../storm/generated/KeyNotFoundException.java   | 406 ++++++++++
 .../backtype/storm/generated/LSTopoHistory.java | 805 +++++++++++++++++++
 .../storm/generated/LSTopoHistoryList.java      | 460 +++++++++++
 .../storm/generated/ListBlobsResult.java        | 556 +++++++++++++
 .../storm/generated/ReadableBlobMeta.java       | 510 ++++++++++++
 .../storm/generated/SettableBlobMeta.java       | 460 +++++++++++
 .../storm/generated/TopologyHistoryInfo.java    | 461 +++++++++++
 .../storm/generated/WorkerResources.java        | 605 ++++++++++++++
 .../backtype/storm/scheduler/WorkerSlot.java    |  24 +
 storm-core/src/storm.thrift                     |   7 +
 .../test/clj/backtype/storm/cluster_test.clj    |   4 +-
 .../test/clj/backtype/storm/supervisor_test.clj | 113 +--
 26 files changed, 8425 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index 27336f0..4d56900 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -14,7 +14,7 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.converter
-  (:import [backtype.storm.generated SupervisorInfo NodeInfo Assignment
+  (:import [backtype.storm.generated SupervisorInfo NodeInfo Assignment WorkerResources
             StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
             TopologyActionOptions DebugOptions])
   (:use [backtype.storm util stats log])
@@ -47,19 +47,29 @@
       (if-let [res-map (.get_resources_map supervisor-info)] (into {} res-map)))))
 
 (defn thriftify-assignment [assignment]
-  (doto (Assignment.)
-    (.set_master_code_dir (:master-code-dir assignment))
-    (.set_node_host (:node->host assignment))
-    (.set_executor_node_port (map-val
-                               (fn [node+port]
-                                 (NodeInfo. (first node+port) (set (map long (rest node+port)))))
-                               (map-key #(map long %)
-                                 (:executor->node+port assignment))))
-    (.set_executor_start_time_secs
-      (map-val
-        long
-        (map-key #(map long %)
-          (:executor->start-time-secs assignment))))))
+  (let [thrift-assignment (doto (Assignment.)
+                            (.set_master_code_dir (:master-code-dir assignment))
+                            (.set_node_host (:node->host assignment))
+                            (.set_executor_node_port (into {}
+                                                           (map (fn [[k v]]
+                                                                  [(map long k)
+                                                                   (NodeInfo. (first v) (set (map long (rest v))))])
+                                                                (:executor->node+port assignment))))
+                            (.set_executor_start_time_secs
+                              (into {}
+                                    (map (fn [[k v]]
+                                           [(map long k) (long v)])
+                                         (:executor->start-time-secs assignment)))))]
+    (if (:worker->resources assignment)
+      (.set_worker_resources thrift-assignment (into {} (map
+                                                          (fn [[node+port resources]]
+                                                            [(NodeInfo. (first node+port) (set (map long (rest node+port))))
+                                                             (doto (WorkerResources.)
+                                                               (.set_mem_on_heap (first resources))
+                                                               (.set_mem_off_heap (second resources))
+                                                               (.set_cpu (last resources)))])
+                                                          (:worker->resources assignment)))))
+    thrift-assignment))
 
 (defn clojurify-executor->node_port [executor->node_port]
   (into {}
@@ -71,6 +81,15 @@
           (into [] list-of-executors)) ; list of executors must be coverted to clojure vector to ensure it is sortable.
         executor->node_port))))
 
+(defn clojurify-worker->resources [worker->resources]
+  "convert worker info to be [node, port]
+   convert resources to be mem_on_heap mem_off_heap cpu]"
+  (into {} (map
+             (fn [[nodeInfo resources]]
+               [(concat [(.get_node nodeInfo)] (.get_port nodeInfo))
+                [(.get_mem_on_heap resources) (.get_mem_off_heap resources) (.get_cpu resources)]])
+             worker->resources)))
+
 (defn clojurify-assignment [^Assignment assignment]
   (if assignment
     (backtype.storm.daemon.common.Assignment.
@@ -78,7 +97,8 @@
       (into {} (.get_node_host assignment))
       (clojurify-executor->node_port (into {} (.get_executor_node_port assignment)))
       (map-key (fn [executor] (into [] executor))
-        (into {} (.get_executor_start_time_secs assignment))))))
+        (into {} (.get_executor_start_time_secs assignment)))
+      (clojurify-worker->resources (into {} (.get_worker_resources assignment))))))
 
 (defn convert-to-symbol-from-status [status]
   (condp = status

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/clj/backtype/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index a1b6241..1a2e932 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -51,7 +51,7 @@
 ;; the task id is the virtual port
 ;; node->host is here so that tasks know who to talk to just from assignment
 ;; this avoid situation where node goes down and task doesn't know what to do information-wise
-(defrecord Assignment [master-code-dir node->host executor->node+port executor->start-time-secs])
+(defrecord Assignment [master-code-dir node->host executor->node+port executor->start-time-secs worker->resources])
 
 
 ;; component->executors is a map from spout/bolt id to number of executors for that component

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index eee7bac..4c3d1c1 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -614,8 +614,42 @@
     (count (.getSlots scheduler-assignment))
     0 ))
 
+(defn convert-assignments-to-worker->resources [new-scheduler-assignments]
+  "convert {topology-id -> SchedulerAssignment} to
+           {topology-id -> {[node port] [mem-on-heap mem-off-heap cpu]}}
+   Make sure this can deal with other non-RAS schedulers
+   later we may further support map-for-any-resources"
+  (map-val (fn [^SchedulerAssignment assignment]
+             (->> assignment
+                  .getExecutorToSlot
+                  .values
+                  (#(into {} (for [^WorkerSlot slot %]
+                              {[(.getNodeId slot) (.getPort slot)]
+                               [(.getAllocatedMemOnHeap slot) (.getAllocatedMemOffHeap slot) (.getAllocatedCpu slot)]
+                               })))))
+           new-scheduler-assignments))
+
+(defn compute-new-topology->executor->node+port [new-scheduler-assignments existing-assignments]
+  (let [new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)]
+    ;; print some useful information.
+    (doseq [[topology-id executor->node+port] new-topology->executor->node+port
+            :let [old-executor->node+port (-> topology-id
+                                              existing-assignments
+                                              :executor->node+port)
+                  reassignment (filter (fn [[executor node+port]]
+                                         (and (contains? old-executor->node+port executor)
+                                              (not (= node+port (old-executor->node+port executor)))))
+                                       executor->node+port)]]
+      (when-not (empty? reassignment)
+        (let [new-slots-cnt (count (set (vals executor->node+port)))
+              reassign-executors (keys reassignment)]
+          (log-message "Reassigning " topology-id " to " new-slots-cnt " slots")
+          (log-message "Reassign executors: " (vec reassign-executors)))))
+
+    new-topology->executor->node+port))
+
 ;; public so it can be mocked out
-(defn compute-new-topology->executor->node+port [nimbus existing-assignments topologies scratch-topology-id]
+(defn compute-new-scheduler-assignments [nimbus existing-assignments topologies scratch-topology-id]
   (let [conf (:conf nimbus)
         storm-cluster-state (:storm-cluster-state nimbus)
         topology->executors (compute-topology->executors nimbus (keys existing-assignments))
@@ -638,16 +672,14 @@
                                            .getTopologies
                                            (map (memfn getId))
                                            (filter (fn [t]
-                                                      (let [alle (get topology->executors t)
-                                                            alivee (get topology->alive-executors t)]
-                                                            (or (empty? alle)
-                                                                (not= alle alivee)
-                                                                (< (-> topology->scheduler-assignment
-                                                                       (get t)
-                                                                       num-used-workers )
-                                                                   (-> topologies (.getById t) .getNumWorkers)
-                                                                   ))
-                                                            ))))
+                                                     (let [alle (get topology->executors t)
+                                                           alivee (get topology->alive-executors t)]
+                                                       (or (empty? alle)
+                                                           (not= alle alivee)
+                                                           (< (-> topology->scheduler-assignment
+                                                                  (get t)
+                                                                  num-used-workers )
+                                                              (-> topologies (.getById t) .getNumWorkers)))))))
         all-scheduling-slots (->> (all-scheduling-slots nimbus topologies missing-assignment-topologies)
                                   (map (fn [[node-id port]] {node-id #{port}}))
                                   (apply merge-with set/union))
@@ -658,26 +690,8 @@
         ;; call scheduler.schedule to schedule all the topologies
         ;; the new assignments for all the topologies are in the cluster object.
         _ (.schedule (:scheduler nimbus) topologies cluster)
-        new-scheduler-assignments (.getAssignments cluster)
-        ;; add more information to convert SchedulerAssignment to Assignment
-        new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)]
-    (reset! (:id->sched-status nimbus) (.getStatusMap cluster))
-    ;; print some useful information.
-    (doseq [[topology-id executor->node+port] new-topology->executor->node+port
-            :let [old-executor->node+port (-> topology-id
-                                          existing-assignments
-                                          :executor->node+port)
-                  reassignment (filter (fn [[executor node+port]]
-                                         (and (contains? old-executor->node+port executor)
-                                              (not (= node+port (old-executor->node+port executor)))))
-                                       executor->node+port)]]
-      (when-not (empty? reassignment)
-        (let [new-slots-cnt (count (set (vals executor->node+port)))
-              reassign-executors (keys reassignment)]
-          (log-message "Reassigning " topology-id " to " new-slots-cnt " slots")
-          (log-message "Reassign executors: " (vec reassign-executors)))))
-
-    new-topology->executor->node+port))
+        _ (reset! (:id->sched-status nimbus) (.getStatusMap cluster))]
+    (.getAssignments cluster)))
 
 (defn changed-executors [executor->node+port new-executor->node+port]
   (let [executor->node+port (if executor->node+port (sort executor->node+port) nil)
@@ -732,14 +746,16 @@
                                         (when (or (nil? scratch-topology-id) (not= tid scratch-topology-id))
                                           {tid (.assignment-info storm-cluster-state tid nil)})))
         ;; make the new assignments for topologies
-        topology->executor->node+port (compute-new-topology->executor->node+port
+        new-scheduler-assignments (compute-new-scheduler-assignments
                                        nimbus
                                        existing-assignments
                                        topologies
                                        scratch-topology-id)
 
-        topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
+        topology->executor->node+port (compute-new-topology->executor->node+port new-scheduler-assignments existing-assignments)
 
+        topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
+        new-assigned-worker->resources (convert-assignments-to-worker->resources new-scheduler-assignments)
         now-secs (current-time-secs)
 
         basic-supervisor-details-map (basic-supervisor-details-map storm-cluster-state)
@@ -760,12 +776,14 @@
                                                                 (into {}
                                                                       (for [id reassign-executors]
                                                                         [id now-secs]
-                                                                        )))]]
+                                                                        )))
+                                              worker->resources (get new-assigned-worker->resources topology-id)]]
                                    {topology-id (Assignment.
                                                  (master-stormdist-root conf topology-id)
                                                  (select-keys all-node->host all-nodes)
                                                  executor->node+port
-                                                 start-times)}))]
+                                                 start-times
+                                                 worker->resources)}))]
 
     ;; tasks figure out what tasks to talk to by looking at topology at runtime
     ;; only log/set when there's been a change to the assignment

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 4b2af2e..d87cc90 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -65,21 +65,24 @@
 
 (defn- read-my-executors [assignments-snapshot storm-id assignment-id]
   (let [assignment (get assignments-snapshot storm-id)
+        my-slots-resources (into {} 
+                                 (filter (fn [[[node _] _]] (= node assignment-id))
+                                         (:worker->resources assignment)))
         my-executors (filter (fn [[_ [node _]]] (= node assignment-id))
-                           (:executor->node+port assignment))
+                             (:executor->node+port assignment))
         port-executors (apply merge-with
-                          concat
-                          (for [[executor [_ port]] my-executors]
-                            {port [executor]}
-                            ))]
+                              concat
+                              (for [[executor [_ port]] my-executors]
+                                {port [executor]}
+                                ))]
     (into {} (for [[port executors] port-executors]
                ;; need to cast to int b/c it might be a long (due to how yaml parses things)
                ;; doall is to avoid serialization/deserialization problems with lazy seqs
-               [(Integer. port) (mk-local-assignment storm-id (doall executors))]
+               [(Integer. port) (mk-local-assignment-with-resources storm-id (doall executors) (get my-slots-resources [assignment-id port]))]
                ))))
 
 (defn- read-assignments
-  "Returns map from port to struct containing :storm-id and :executors"
+  "Returns map from port to struct containing :storm-id, :executors and :resources"
   ([assignments-snapshot assignment-id]
      (->> (dofor [sid (keys assignments-snapshot)] (read-my-executors assignments-snapshot sid assignment-id))
           (apply merge-with (fn [& ignored] (throw-runtime "Should not have multiple topologies assigned to one port")))))
@@ -453,8 +456,7 @@
                                            (:sync-retry supervisor))
           new-assignment (->> all-assignment
                               (filter-key #(.confirmAssigned isupervisor %)))
-          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)
-          ]
+          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)]
       (log-debug "Synchronizing supervisor")
       (log-debug "Storm code map: " storm-code-map)
       (log-debug "Downloaded storm ids: " downloaded-storm-ids)

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/clj/backtype/storm/local_state.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/local_state.clj b/storm-core/src/clj/backtype/storm/local_state.clj
index 41e3675..745d3b0 100644
--- a/storm-core/src/clj/backtype/storm/local_state.clj
+++ b/storm-core/src/clj/backtype/storm/local_state.clj
@@ -19,7 +19,8 @@
             InvalidTopologyException GlobalStreamId
             LSSupervisorId LSApprovedWorkers
             LSSupervisorAssignments LocalAssignment
-            ExecutorInfo LSWorkerHeartbeat])
+            ExecutorInfo
+            WorkerResources])
   (:import [backtype.storm.utils LocalState]))
 
 (def LS-WORKER-HEARTBEAT "worker-heartbeat")
@@ -59,8 +60,18 @@
       [(.get_task_start exec-info) (.get_task_end exec-info)])))
 
 (defn ->LocalAssignment
-  [{storm-id :storm-id executors :executors}]
-  (LocalAssignment. storm-id (->ExecutorInfo-list executors)))
+  [{storm-id :storm-id executors :executors resources :resources}]
+  (let [assignment (LocalAssignment. storm-id (->ExecutorInfo-list executors))]
+    (if resources (.set_resources assignment
+                                  (doto (WorkerResources. )
+                                    (.set_mem_on_heap (first resources))
+                                    (.set_mem_off_heap (second resources))
+                                    (.set_cpu (last resources)))))
+    assignment))
+
+(defn mk-local-assignment-with-resources
+  [storm-id executors resources]
+  {:storm-id storm-id :executors executors :resources resources})
 
 (defn mk-local-assignment
   [storm-id executors]

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/clj/backtype/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index b0c6637..e1a928c 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -281,22 +281,39 @@
     (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
   (.submitTopologyWithOpts nimbus storm-name nil (to-json conf) topology submit-opts))
 
-(defn mocked-compute-new-topology->executor->node+port [storm-name executor->node+port]
-  (fn [nimbus existing-assignments topologies scratch-topology-id]
-    (let [topology (.getByName topologies storm-name)
-          topology-id (.getId topology)
+(defn mocked-convert-assignments-to-worker->resources [storm-cluster-state storm-name worker->resources]
+  (fn [existing-assignments]
+    (let [topology-id (common/get-storm-id storm-cluster-state storm-name)
+          existing-assignments (into {} (for [[tid assignment] existing-assignments]
+                                          {tid (:worker->resources assignment)}))
+          new-assignments (assoc existing-assignments topology-id worker->resources)]
+      new-assignments)))
+
+(defn mocked-compute-new-topology->executor->node+port [storm-cluster-state storm-name executor->node+port]
+  (fn [new-scheduler-assignments existing-assignments]
+    (let [topology-id (common/get-storm-id storm-cluster-state storm-name)
           existing-assignments (into {} (for [[tid assignment] existing-assignments]
                                           {tid (:executor->node+port assignment)}))
           new-assignments (assoc existing-assignments topology-id executor->node+port)]
       new-assignments)))
 
+(defn mocked-compute-new-scheduler-assignments []
+  (fn [nimbus existing-assignments topologies scratch-topology-id]
+    existing-assignments))
+
 (defn submit-mocked-assignment
-  [nimbus storm-name conf topology task->component executor->node+port]
+  [nimbus storm-cluster-state storm-name conf topology task->component executor->node+port worker->resources]
   (with-var-roots [common/storm-task-info (fn [& ignored] task->component)
+                   nimbus/compute-new-scheduler-assignments (mocked-compute-new-scheduler-assignments)
+                   nimbus/convert-assignments-to-worker->resources (mocked-convert-assignments-to-worker->resources
+                                                          storm-cluster-state
+                                                          storm-name
+                                                          worker->resources)
                    nimbus/compute-new-topology->executor->node+port (mocked-compute-new-topology->executor->node+port
+                                                                      storm-cluster-state
                                                                       storm-name
                                                                       executor->node+port)]
-                  (submit-local-topology nimbus storm-name conf topology)))
+    (submit-local-topology nimbus storm-name conf topology)))
 
 (defn mk-capture-launch-fn [capture-atom]
   (fn [supervisor storm-id port worker-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/17d70d3b/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/AccessControl.java b/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
new file mode 100644
index 0000000..e0ff680
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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 Integer.valueOf(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 = backtype.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 = backtype.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/17d70d3b/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java b/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java
new file mode 100644
index 0000000..e4737ed
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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/17d70d3b/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java b/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java
new file mode 100644
index 0000000..b803b44
--- /dev/null
+++ b/storm-core/src/jvm/backtype/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.2)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.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.2)", date = "2015-8-5")
+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);
+    }
+  }
+
+}
+


[18/18] storm git commit: Adding STORM-1093 to the changelog

Posted by kn...@apache.org.
Adding STORM-1093 to the changelog


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

Branch: refs/heads/master
Commit: d23bd3ed244d7cddfe7e0b2adc2ef7871b055dfb
Parents: c88a4f6
Author: Kyle Nusbaum <Ky...@gmail.com>
Authored: Mon Oct 19 15:06:24 2015 -0500
Committer: Kyle Nusbaum <Ky...@gmail.com>
Committed: Mon Oct 19 15:06:24 2015 -0500

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


http://git-wip-us.apache.org/repos/asf/storm/blob/d23bd3ed/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index ecff1a2..813232d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-1093: Launching Workers with resources specified in resource-aware schedulers
  * STORM-1102: Add a default flush interval for HiveBolt
  * STORM-1112: Add executor id to the thread name of the executor thread for debug
  * STORM-1079: Batch Puts to HBase