You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2015/03/18 20:39:08 UTC

[01/16] storm git commit: STORM-634: Converting SupervisorInfo, Assignment, StormBase, TopologyStatus, ZKWorkerHeartbeat, ErrorInfo, Credentials to thrift and defaulting the serialization delegate to thrift serialization. Added class as a param to serializatio

Repository: storm
Updated Branches:
  refs/heads/master 1ea378d78 -> bb8d48da2


http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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 1bbaf37..e15cf1d 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -44,6 +44,26 @@ class TopologyInitialStatus:
     "INACTIVE": 2,
   }
 
+class TopologyStatus:
+  ACTIVE = 1
+  INACTIVE = 2
+  REBALANCING = 3
+  KILLED = 4
+
+  _VALUES_TO_NAMES = {
+    1: "ACTIVE",
+    2: "INACTIVE",
+    3: "REBALANCING",
+    4: "KILLED",
+  }
+
+  _NAMES_TO_VALUES = {
+    "ACTIVE": 1,
+    "INACTIVE": 2,
+    "REBALANCING": 3,
+    "KILLED": 4,
+  }
+
 
 class JavaObjectArg:
   """
@@ -3034,6 +3054,7 @@ class ExecutorStats:
    - emitted
    - transferred
    - specific
+   - rate
   """
 
   thrift_spec = (
@@ -3041,15 +3062,17 @@ class ExecutorStats:
     (1, TType.MAP, 'emitted', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 1
     (2, TType.MAP, 'transferred', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 2
     (3, TType.STRUCT, 'specific', (ExecutorSpecificStats, ExecutorSpecificStats.thrift_spec), None, ), # 3
+    (4, TType.DOUBLE, 'rate', None, None, ), # 4
   )
 
   def __hash__(self):
-    return 0 + hash(self.emitted) + hash(self.transferred) + hash(self.specific)
+    return 0 + hash(self.emitted) + hash(self.transferred) + hash(self.specific) + hash(self.rate)
 
-  def __init__(self, emitted=None, transferred=None, specific=None,):
+  def __init__(self, emitted=None, transferred=None, specific=None, rate=None,):
     self.emitted = emitted
     self.transferred = transferred
     self.specific = specific
+    self.rate = rate
 
   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:
@@ -3100,6 +3123,11 @@ class ExecutorStats:
           self.specific.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.DOUBLE:
+          self.rate = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3138,6 +3166,10 @@ class ExecutorStats:
       oprot.writeFieldBegin('specific', TType.STRUCT, 3)
       self.specific.write(oprot)
       oprot.writeFieldEnd()
+    if self.rate is not None:
+      oprot.writeFieldBegin('rate', TType.DOUBLE, 4)
+      oprot.writeDouble(self.rate)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3148,6 +3180,8 @@ class ExecutorStats:
       raise TProtocol.TProtocolException(message='Required field transferred is unset!')
     if self.specific is None:
       raise TProtocol.TProtocolException(message='Required field specific is unset!')
+    if self.rate is None:
+      raise TProtocol.TProtocolException(message='Required field rate is unset!')
     return
 
 
@@ -4383,6 +4417,764 @@ class SubmitOptions:
   def __ne__(self, other):
     return not (self == other)
 
+class SupervisorInfo:
+  """
+  Attributes:
+   - time_secs
+   - hostname
+   - assignment_id
+   - used_ports
+   - meta
+   - scheduler_meta
+   - uptime_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'time_secs', None, None, ), # 1
+    (2, TType.STRING, 'hostname', None, None, ), # 2
+    (3, TType.STRING, 'assignment_id', None, None, ), # 3
+    (4, TType.LIST, 'used_ports', (TType.I64,None), None, ), # 4
+    (5, TType.LIST, 'meta', (TType.I64,None), None, ), # 5
+    (6, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 6
+    (7, TType.I64, 'uptime_secs', None, None, ), # 7
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.time_secs) + hash(self.hostname) + hash(self.assignment_id) + hash(self.used_ports) + hash(self.meta) + hash(self.scheduler_meta) + hash(self.uptime_secs)
+
+  def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None,):
+    self.time_secs = time_secs
+    self.hostname = hostname
+    self.assignment_id = assignment_id
+    self.used_ports = used_ports
+    self.meta = meta
+    self.scheduler_meta = scheduler_meta
+    self.uptime_secs = uptime_secs
+
+  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.I64:
+          self.time_secs = iprot.readI64();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.hostname = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.assignment_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.used_ports = []
+          (_etype304, _size301) = iprot.readListBegin()
+          for _i305 in xrange(_size301):
+            _elem306 = iprot.readI64();
+            self.used_ports.append(_elem306)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.LIST:
+          self.meta = []
+          (_etype310, _size307) = iprot.readListBegin()
+          for _i311 in xrange(_size307):
+            _elem312 = iprot.readI64();
+            self.meta.append(_elem312)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.MAP:
+          self.scheduler_meta = {}
+          (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin() 
+          for _i317 in xrange(_size313):
+            _key318 = iprot.readString().decode('utf-8')
+            _val319 = iprot.readString().decode('utf-8')
+            self.scheduler_meta[_key318] = _val319
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I64:
+          self.uptime_secs = iprot.readI64();
+        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('SupervisorInfo')
+    if self.time_secs is not None:
+      oprot.writeFieldBegin('time_secs', TType.I64, 1)
+      oprot.writeI64(self.time_secs)
+      oprot.writeFieldEnd()
+    if self.hostname is not None:
+      oprot.writeFieldBegin('hostname', TType.STRING, 2)
+      oprot.writeString(self.hostname.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.assignment_id is not None:
+      oprot.writeFieldBegin('assignment_id', TType.STRING, 3)
+      oprot.writeString(self.assignment_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.used_ports is not None:
+      oprot.writeFieldBegin('used_ports', TType.LIST, 4)
+      oprot.writeListBegin(TType.I64, len(self.used_ports))
+      for iter320 in self.used_ports:
+        oprot.writeI64(iter320)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.meta is not None:
+      oprot.writeFieldBegin('meta', TType.LIST, 5)
+      oprot.writeListBegin(TType.I64, len(self.meta))
+      for iter321 in self.meta:
+        oprot.writeI64(iter321)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.scheduler_meta is not None:
+      oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta))
+      for kiter322,viter323 in self.scheduler_meta.items():
+        oprot.writeString(kiter322.encode('utf-8'))
+        oprot.writeString(viter323.encode('utf-8'))
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.uptime_secs is not None:
+      oprot.writeFieldBegin('uptime_secs', TType.I64, 7)
+      oprot.writeI64(self.uptime_secs)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.time_secs is None:
+      raise TProtocol.TProtocolException(message='Required field time_secs is unset!')
+    if self.hostname is None:
+      raise TProtocol.TProtocolException(message='Required field hostname is unset!')
+    return
+
+
+  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 NodeInfo:
+  """
+  Attributes:
+   - node
+   - port
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'node', None, None, ), # 1
+    (2, TType.SET, 'port', (TType.I64,None), None, ), # 2
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.node) + hash(self.port)
+
+  def __init__(self, node=None, port=None,):
+    self.node = node
+    self.port = port
+
+  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.STRING:
+          self.node = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.SET:
+          self.port = set()
+          (_etype327, _size324) = iprot.readSetBegin()
+          for _i328 in xrange(_size324):
+            _elem329 = iprot.readI64();
+            self.port.add(_elem329)
+          iprot.readSetEnd()
+        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('NodeInfo')
+    if self.node is not None:
+      oprot.writeFieldBegin('node', TType.STRING, 1)
+      oprot.writeString(self.node.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.port is not None:
+      oprot.writeFieldBegin('port', TType.SET, 2)
+      oprot.writeSetBegin(TType.I64, len(self.port))
+      for iter330 in self.port:
+        oprot.writeI64(iter330)
+      oprot.writeSetEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.node is None:
+      raise TProtocol.TProtocolException(message='Required field node is unset!')
+    if self.port is None:
+      raise TProtocol.TProtocolException(message='Required field port is unset!')
+    return
+
+
+  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:
+   - master_code_dir
+   - node_host
+   - executor_node_port
+   - executor_start_time_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'master_code_dir', None, None, ), # 1
+    (2, TType.MAP, 'node_host', (TType.STRING,None,TType.STRING,None), {
+    }, ), # 2
+    (3, TType.MAP, 'executor_node_port', (TType.LIST,(TType.I64,None),TType.STRUCT,(NodeInfo, NodeInfo.thrift_spec)), {
+    }, ), # 3
+    (4, TType.MAP, 'executor_start_time_secs', (TType.LIST,(TType.I64,None),TType.I64,None), {
+    }, ), # 4
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.master_code_dir) + hash(self.node_host) + hash(self.executor_node_port) + hash(self.executor_start_time_secs)
+
+  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],):
+    self.master_code_dir = master_code_dir
+    if node_host is self.thrift_spec[2][4]:
+      node_host = {
+    }
+    self.node_host = node_host
+    if executor_node_port is self.thrift_spec[3][4]:
+      executor_node_port = {
+    }
+    self.executor_node_port = executor_node_port
+    if executor_start_time_secs is self.thrift_spec[4][4]:
+      executor_start_time_secs = {
+    }
+    self.executor_start_time_secs = executor_start_time_secs
+
+  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.STRING:
+          self.master_code_dir = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.node_host = {}
+          (_ktype332, _vtype333, _size331 ) = iprot.readMapBegin() 
+          for _i335 in xrange(_size331):
+            _key336 = iprot.readString().decode('utf-8')
+            _val337 = iprot.readString().decode('utf-8')
+            self.node_host[_key336] = _val337
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.MAP:
+          self.executor_node_port = {}
+          (_ktype339, _vtype340, _size338 ) = iprot.readMapBegin() 
+          for _i342 in xrange(_size338):
+            _key343 = []
+            (_etype348, _size345) = iprot.readListBegin()
+            for _i349 in xrange(_size345):
+              _elem350 = iprot.readI64();
+              _key343.append(_elem350)
+            iprot.readListEnd()
+            _val344 = NodeInfo()
+            _val344.read(iprot)
+            self.executor_node_port[_key343] = _val344
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.MAP:
+          self.executor_start_time_secs = {}
+          (_ktype352, _vtype353, _size351 ) = iprot.readMapBegin() 
+          for _i355 in xrange(_size351):
+            _key356 = []
+            (_etype361, _size358) = iprot.readListBegin()
+            for _i362 in xrange(_size358):
+              _elem363 = iprot.readI64();
+              _key356.append(_elem363)
+            iprot.readListEnd()
+            _val357 = iprot.readI64();
+            self.executor_start_time_secs[_key356] = _val357
+          iprot.readMapEnd()
+        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('Assignment')
+    if self.master_code_dir is not None:
+      oprot.writeFieldBegin('master_code_dir', TType.STRING, 1)
+      oprot.writeString(self.master_code_dir.encode('utf-8'))
+      oprot.writeFieldEnd()
+    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 kiter364,viter365 in self.node_host.items():
+        oprot.writeString(kiter364.encode('utf-8'))
+        oprot.writeString(viter365.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 kiter366,viter367 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter366))
+        for iter368 in kiter366:
+          oprot.writeI64(iter368)
+        oprot.writeListEnd()
+        viter367.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 kiter369,viter370 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter369))
+        for iter371 in kiter369:
+          oprot.writeI64(iter371)
+        oprot.writeListEnd()
+        oprot.writeI64(viter370)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.master_code_dir is None:
+      raise TProtocol.TProtocolException(message='Required field master_code_dir is unset!')
+    return
+
+
+  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 TopologyActionOptions:
+  """
+  Attributes:
+   - kill_options
+   - rebalance_options
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'kill_options', (KillOptions, KillOptions.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'rebalance_options', (RebalanceOptions, RebalanceOptions.thrift_spec), None, ), # 2
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.kill_options) + hash(self.rebalance_options)
+
+  def __init__(self, kill_options=None, rebalance_options=None,):
+    self.kill_options = kill_options
+    self.rebalance_options = rebalance_options
+
+  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.STRUCT:
+          self.kill_options = KillOptions()
+          self.kill_options.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.rebalance_options = RebalanceOptions()
+          self.rebalance_options.read(iprot)
+        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('TopologyActionOptions')
+    if self.kill_options is not None:
+      oprot.writeFieldBegin('kill_options', TType.STRUCT, 1)
+      self.kill_options.write(oprot)
+      oprot.writeFieldEnd()
+    if self.rebalance_options is not None:
+      oprot.writeFieldBegin('rebalance_options', TType.STRUCT, 2)
+      self.rebalance_options.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  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 StormBase:
+  """
+  Attributes:
+   - name
+   - status
+   - num_workers
+   - component_executors
+   - launch_time_secs
+   - owner
+   - topology_action_options
+   - prev_status
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.I32, 'status', None, None, ), # 2
+    (3, TType.I32, 'num_workers', None, None, ), # 3
+    (4, TType.MAP, 'component_executors', (TType.STRING,None,TType.I32,None), None, ), # 4
+    (5, TType.I32, 'launch_time_secs', None, None, ), # 5
+    (6, TType.STRING, 'owner', None, None, ), # 6
+    (7, TType.STRUCT, 'topology_action_options', (TopologyActionOptions, TopologyActionOptions.thrift_spec), None, ), # 7
+    (8, TType.I32, 'prev_status', None, None, ), # 8
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.name) + hash(self.status) + hash(self.num_workers) + hash(self.component_executors) + hash(self.launch_time_secs) + hash(self.owner) + hash(self.topology_action_options) + hash(self.prev_status)
+
+  def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None,):
+    self.name = name
+    self.status = status
+    self.num_workers = num_workers
+    self.component_executors = component_executors
+    self.launch_time_secs = launch_time_secs
+    self.owner = owner
+    self.topology_action_options = topology_action_options
+    self.prev_status = prev_status
+
+  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.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.status = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.num_workers = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.MAP:
+          self.component_executors = {}
+          (_ktype373, _vtype374, _size372 ) = iprot.readMapBegin() 
+          for _i376 in xrange(_size372):
+            _key377 = iprot.readString().decode('utf-8')
+            _val378 = iprot.readI32();
+            self.component_executors[_key377] = _val378
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.launch_time_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.owner = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRUCT:
+          self.topology_action_options = TopologyActionOptions()
+          self.topology_action_options.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.I32:
+          self.prev_status = iprot.readI32();
+        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('StormBase')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.status is not None:
+      oprot.writeFieldBegin('status', TType.I32, 2)
+      oprot.writeI32(self.status)
+      oprot.writeFieldEnd()
+    if self.num_workers is not None:
+      oprot.writeFieldBegin('num_workers', TType.I32, 3)
+      oprot.writeI32(self.num_workers)
+      oprot.writeFieldEnd()
+    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 kiter379,viter380 in self.component_executors.items():
+        oprot.writeString(kiter379.encode('utf-8'))
+        oprot.writeI32(viter380)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.launch_time_secs is not None:
+      oprot.writeFieldBegin('launch_time_secs', TType.I32, 5)
+      oprot.writeI32(self.launch_time_secs)
+      oprot.writeFieldEnd()
+    if self.owner is not None:
+      oprot.writeFieldBegin('owner', TType.STRING, 6)
+      oprot.writeString(self.owner.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.topology_action_options is not None:
+      oprot.writeFieldBegin('topology_action_options', TType.STRUCT, 7)
+      self.topology_action_options.write(oprot)
+      oprot.writeFieldEnd()
+    if self.prev_status is not None:
+      oprot.writeFieldBegin('prev_status', TType.I32, 8)
+      oprot.writeI32(self.prev_status)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.name is None:
+      raise TProtocol.TProtocolException(message='Required field name is unset!')
+    if self.status is None:
+      raise TProtocol.TProtocolException(message='Required field status is unset!')
+    if self.num_workers is None:
+      raise TProtocol.TProtocolException(message='Required field num_workers is unset!')
+    return
+
+
+  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 ZKWorkerHeartbeat:
+  """
+  Attributes:
+   - storm_id
+   - executor_stats
+   - time_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'storm_id', None, None, ), # 1
+    (2, TType.MAP, 'executor_stats', (TType.STRUCT,(ExecutorInfo, ExecutorInfo.thrift_spec),TType.STRUCT,(ExecutorStats, ExecutorStats.thrift_spec)), None, ), # 2
+    (3, TType.I32, 'time_secs', None, None, ), # 3
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.storm_id) + hash(self.executor_stats) + hash(self.time_secs)
+
+  def __init__(self, storm_id=None, executor_stats=None, time_secs=None,):
+    self.storm_id = storm_id
+    self.executor_stats = executor_stats
+    self.time_secs = time_secs
+
+  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.STRING:
+          self.storm_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.executor_stats = {}
+          (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin() 
+          for _i385 in xrange(_size381):
+            _key386 = ExecutorInfo()
+            _key386.read(iprot)
+            _val387 = ExecutorStats()
+            _val387.read(iprot)
+            self.executor_stats[_key386] = _val387
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.time_secs = iprot.readI32();
+        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('ZKWorkerHeartbeat')
+    if self.storm_id is not None:
+      oprot.writeFieldBegin('storm_id', TType.STRING, 1)
+      oprot.writeString(self.storm_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    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 kiter388,viter389 in self.executor_stats.items():
+        kiter388.write(oprot)
+        viter389.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.time_secs is not None:
+      oprot.writeFieldBegin('time_secs', TType.I32, 3)
+      oprot.writeI32(self.time_secs)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.storm_id is None:
+      raise TProtocol.TProtocolException(message='Required field storm_id is unset!')
+    if self.executor_stats is None:
+      raise TProtocol.TProtocolException(message='Required field executor_stats is unset!')
+    if self.time_secs is None:
+      raise TProtocol.TProtocolException(message='Required field time_secs is unset!')
+    return
+
+
+  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 DRPCRequest:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index f807b74..3cc0eb9 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -193,6 +193,7 @@ struct ExecutorStats {
   1: required map<string, map<string, i64>> emitted;
   2: required map<string, map<string, i64>> transferred;
   3: required ExecutorSpecificStats specific;
+  4: required double rate;
 }
 
 struct ExecutorInfo {
@@ -243,6 +244,56 @@ struct SubmitOptions {
   2: optional Credentials creds;
 }
 
+struct SupervisorInfo {
+    1: required i64 time_secs;
+    2: required string hostname;
+    3: optional string assignment_id;
+    4: optional list<i64> used_ports;
+    5: optional list<i64> meta;
+    6: optional map<string, string> scheduler_meta;
+    7: optional i64 uptime_secs;
+}
+struct NodeInfo {
+    1: required string node;
+    2: required set<i64> port;
+}
+
+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 = {};
+}
+
+enum TopologyStatus {
+    ACTIVE = 1,
+    INACTIVE = 2,
+    REBALANCING = 3,
+    KILLED = 4
+}
+
+union TopologyActionOptions {
+    1: optional KillOptions kill_options;
+    2: optional RebalanceOptions rebalance_options;
+}
+
+struct StormBase {
+    1: required string name;
+    2: required TopologyStatus status;
+    3: required i32 num_workers;
+    4: optional map<string, i32> component_executors;
+    5: optional i32 launch_time_secs;
+    6: optional string owner;
+    7: optional TopologyActionOptions topology_action_options;
+    8: optional TopologyStatus prev_status;//currently only used during rebalance action.
+}
+
+struct ZKWorkerHeartbeat {
+    1: required string storm_id;
+    2: required map<ExecutorInfo,ExecutorStats> executor_stats;
+    3: required i32 time_secs;
+}
+
 service Nimbus {
   void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
   void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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 7ed1028..98eae68 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -25,7 +25,7 @@
   (:require [conjure.core])
   (:use [conjure core])
   (:use [clojure test])
-  (:use [backtype.storm cluster config util testing]))
+  (:use [backtype.storm cluster config util testing thrift log]))
 
 (defn mk-config [zk-port]
   (merge (read-storm-config)
@@ -168,10 +168,10 @@
 (deftest test-storm-cluster-state-basics
   (with-inprocess-zookeeper zk-port
     (let [state (mk-storm-state zk-port)
-          assignment1 (Assignment. "/aaa" {} {1 [2 2002 1]} {})
-          assignment2 (Assignment. "/aaa" {} {1 [2 2002]} {})
-          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "")
-          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "")]
+          assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {})
+          assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {})
+          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil)
+          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil)]
       (is (= [] (.assignments state nil)))
       (.set-assignment! state "storm1" assignment1)
       (is (= assignment1 (.assignment-info state "storm1" nil)))
@@ -242,12 +242,15 @@
 (deftest test-supervisor-state
   (with-inprocess-zookeeper zk-port
     (let [state1 (mk-storm-state zk-port)
-          state2 (mk-storm-state zk-port)]
+          state2 (mk-storm-state zk-port)
+          supervisor-info1 (SupervisorInfo. 10 "hostname-1" "id1" [1 2] [] {} 1000 )
+          supervisor-info2 (SupervisorInfo. 10 "hostname-2" "id2" [1 2] [] {} 1000 )
+          ]
       (is (= [] (.supervisors state1 nil)))
-      (.supervisor-heartbeat! state2 "2" {:a 1})
-      (.supervisor-heartbeat! state1 "1" {})
-      (is (= {:a 1} (.supervisor-info state1 "2")))
-      (is (= {} (.supervisor-info state1 "1")))
+      (.supervisor-heartbeat! state2 "2" supervisor-info2)
+      (.supervisor-heartbeat! state1 "1" supervisor-info1)
+      (is (= supervisor-info2 (.supervisor-info state1 "2")))
+      (is (= supervisor-info1 (.supervisor-info state1 "1")))
       (is (= #{"1" "2"} (set (.supervisors state1 nil))))
       (is (= #{"1" "2"} (set (.supervisors state2 nil))))
       (.disconnect state2)
@@ -255,8 +258,6 @@
       (.disconnect state1)
       )))
 
-
-
 (deftest test-cluster-authentication
   (with-inprocess-zookeeper zk-port
     (let [builder (Mockito/mock CuratorFrameworkFactory$Builder)

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index efdad25..7671f58 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -15,7 +15,7 @@
 ;; limitations under the License.
 (ns backtype.storm.nimbus-test
   (:use [clojure test])
-  (:require [backtype.storm [util :as util]])
+  (:require [backtype.storm [util :as util] [stats :as stats]])
   (:require [backtype.storm.daemon [nimbus :as nimbus]])
   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
   (:import [backtype.storm.scheduler INimbus])
@@ -113,7 +113,7 @@
         curr-beat (.get-worker-heartbeat state storm-id node port)
         stats (:executor-stats curr-beat)]
     (.worker-heartbeat! state storm-id node port
-      {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor nil})}
+      {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
       )))
 
 (defn slot-assignments [cluster storm-id]
@@ -486,7 +486,7 @@
       (bind [executor-id1 executor-id2]  (topology-executors cluster storm-id))
       (bind ass1 (executor-assignment cluster storm-id executor-id1))
       (bind ass2 (executor-assignment cluster storm-id executor-id2))
-      
+
       (advance-cluster-time cluster 59)
       (do-executor-heartbeat cluster storm-id executor-id1)
       (do-executor-heartbeat cluster storm-id executor-id2)

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/test/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegateTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegateTest.java b/storm-core/test/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegateTest.java
index ce49a8a..9026ec3 100644
--- a/storm-core/test/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegateTest.java
+++ b/storm-core/test/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegateTest.java
@@ -41,7 +41,7 @@ public class GzipBridgeSerializationDelegateTest {
 
         byte[] serialized = new GzipSerializationDelegate().serialize(pojo);
 
-        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized);
+        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized, TestPojo.class);
 
         assertEquals(pojo2.name, pojo.name);
         assertEquals(pojo2.age, pojo.age);
@@ -55,7 +55,7 @@ public class GzipBridgeSerializationDelegateTest {
 
         byte[] serialized = new GzipBridgeSerializationDelegate().serialize(pojo);
 
-        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized);
+        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized, TestPojo.class);
 
         assertEquals(pojo2.name, pojo.name);
         assertEquals(pojo2.age, pojo.age);
@@ -69,7 +69,7 @@ public class GzipBridgeSerializationDelegateTest {
 
         byte[] serialized = new DefaultSerializationDelegate().serialize(pojo);
 
-        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized);
+        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized, TestPojo.class);
 
         assertEquals(pojo2.name, pojo.name);
         assertEquals(pojo2.age, pojo.age);

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java b/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java
new file mode 100644
index 0000000..ef17017
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.serialization;
+
+import backtype.storm.generated.ErrorInfo;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.Serializable;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class ThriftBridgeSerializationDelegateTest {
+
+    SerializationDelegate testDelegate;
+
+    @Before
+    public void setUp() throws Exception {
+        testDelegate = new ThriftSerializationDelegateBridge();
+        testDelegate.prepare(null);
+    }
+
+    @Test
+    public void testNonThriftInstance() throws Exception {
+        TestPojo pojo = new TestPojo();
+        pojo.name = "foo";
+        pojo.age = 100;
+
+        byte[] serialized = new DefaultSerializationDelegate().serialize(pojo);
+
+        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized, TestPojo.class);
+
+        assertEquals(pojo2.name, pojo.name);
+        assertEquals(pojo2.age, pojo.age);
+
+        serialized = testDelegate.serialize(pojo);
+        pojo2 = (TestPojo) new DefaultSerializationDelegate().deserialize(serialized, Serializable.class);
+        assertEquals(pojo2.name, pojo.name);
+        assertEquals(pojo2.age, pojo.age);
+    }
+
+    @Test
+    public void testThriftInstance() throws Exception {
+        ErrorInfo errorInfo = new ErrorInfo();
+        errorInfo.set_error("error");
+        errorInfo.set_error_time_secs(1);
+        errorInfo.set_host("host");
+        errorInfo.set_port(1);
+
+        byte[] serialized = new ThriftSerializationDelegate().serialize(errorInfo);
+        ErrorInfo errorInfo2 = testDelegate.deserialize(serialized, ErrorInfo.class);
+        assertEquals(errorInfo, errorInfo2);
+
+        serialized = testDelegate.serialize(errorInfo);
+        errorInfo2 = new ThriftSerializationDelegate().deserialize(serialized, ErrorInfo.class);
+        assertEquals(errorInfo, errorInfo2);
+    }
+
+    static class TestPojo implements Serializable {
+        String name;
+        int age;
+    }
+}


[08/16] storm git commit: Merge remote-tracking branch 'upstream/master' into ru

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into ru

Conflicts:
	storm-core/src/clj/backtype/storm/bootstrap.clj
	storm-core/src/clj/backtype/storm/config.clj
	storm-core/src/clj/backtype/storm/daemon/executor.clj
	storm-core/src/clj/backtype/storm/daemon/nimbus.clj
	storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
	storm-core/src/py/storm/ttypes.py


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

Branch: refs/heads/master
Commit: 5b67b571fe45e14fd52b835bc1de056538de1e70
Parents: f23b3c8 5eff2e7
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Mar 5 17:05:33 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Mar 5 17:05:33 2015 -0800

----------------------------------------------------------------------
 .gitignore                                      |    4 +-
 BYLAWS.md                                       |   31 +-
 CHANGELOG.md                                    |   19 +
 DEVELOPER.md                                    |   24 +-
 README.markdown                                 |    3 +
 SECURITY.md                                     |   10 +-
 STORM-UI-REST-API.md                            |   45 +-
 bin/storm-config.cmd                            |   14 +-
 bin/storm.cmd                                   |   33 +-
 conf/defaults.yaml                              |    2 +-
 docs/documentation/Powered-By.md                |   83 +
 docs/documentation/Trident-API-Overview.md      |    2 +-
 docs/images/architecture.png                    |  Bin 0 -> 69825 bytes
 docs/images/architecture.svg                    | 1458 +++
 docs/images/bolt.png                            |  Bin 0 -> 24796 bytes
 docs/images/bolt.svg                            |  743 ++
 docs/images/security.png                        |  Bin 0 -> 72415 bytes
 docs/images/security.svg                        | 1779 ++++
 docs/images/spout.png                           |  Bin 0 -> 22911 bytes
 docs/images/spout.svg                           |  833 ++
 docs/images/storm.svg                           | 1326 +++
 docs/images/topology.png                        |  Bin 59837 -> 64740 bytes
 docs/images/topology.svg                        | 1044 ++
 docs/images/topology_dark.png                   |  Bin 0 -> 49692 bytes
 docs/images/topology_dark.svg                   | 1101 +++
 .../jvm/storm/starter/BasicDRPCTopology.java    |    3 +-
 external/storm-hdfs/pom.xml                     |   18 +-
 .../storm/hdfs/bolt/HdfsFileTopology.java       |    6 +-
 .../storm/hdfs/bolt/SequenceFileTopology.java   |    4 +-
 .../storm/hdfs/trident/TridentFileTopology.java |    2 +-
 .../hdfs/trident/TridentSequenceTopology.java   |    6 +-
 external/storm-hive/README.md                   |  113 +
 external/storm-hive/pom.xml                     |  143 +
 .../org/apache/storm/hive/bolt/HiveBolt.java    |  289 +
 .../bolt/mapper/DelimitedRecordHiveMapper.java  |  143 +
 .../storm/hive/bolt/mapper/HiveMapper.java      |   81 +
 .../hive/bolt/mapper/JsonRecordHiveMapper.java  |  132 +
 .../apache/storm/hive/common/HiveOptions.java   |  146 +
 .../org/apache/storm/hive/common/HiveUtils.java |   76 +
 .../apache/storm/hive/common/HiveWriter.java    |  420 +
 .../apache/storm/hive/trident/HiveState.java    |  306 +
 .../storm/hive/trident/HiveStateFactory.java    |   31 +
 .../apache/storm/hive/trident/HiveUpdater.java  |   14 +
 .../apache/storm/hive/bolt/HiveSetupUtil.java   |  220 +
 .../apache/storm/hive/bolt/HiveTopology.java    |  150 +
 .../hive/bolt/HiveTopologyPartitioned.java      |  153 +
 .../apache/storm/hive/bolt/TestHiveBolt.java    |  330 +
 .../storm/hive/common/TestHiveWriter.java       |  193 +
 .../storm/hive/trident/TridentHiveTopology.java |  190 +
 external/storm-jdbc/LICENSE                     |  202 +
 external/storm-jdbc/README.md                   |  240 +
 external/storm-jdbc/pom.xml                     |  125 +
 .../storm/jdbc/bolt/AbstractJdbcBolt.java       |   57 +
 .../apache/storm/jdbc/bolt/JdbcInsertBolt.java  |   71 +
 .../apache/storm/jdbc/bolt/JdbcLookupBolt.java  |   76 +
 .../org/apache/storm/jdbc/common/Column.java    |  111 +
 .../apache/storm/jdbc/common/JdbcClient.java    |  228 +
 .../java/org/apache/storm/jdbc/common/Util.java |   75 +
 .../storm/jdbc/mapper/JdbcLookupMapper.java     |   26 +
 .../apache/storm/jdbc/mapper/JdbcMapper.java    |   33 +
 .../jdbc/mapper/SimpleJdbcLookupMapper.java     |   46 +
 .../storm/jdbc/mapper/SimpleJdbcMapper.java     |   92 +
 .../storm/jdbc/trident/state/JdbcQuery.java     |   40 +
 .../storm/jdbc/trident/state/JdbcState.java     |  145 +
 .../jdbc/trident/state/JdbcStateFactory.java    |   40 +
 .../storm/jdbc/trident/state/JdbcUpdater.java   |   32 +
 .../storm/jdbc/common/JdbcClientTest.java       |   88 +
 .../org/apache/storm/jdbc/common/UtilTest.java  |   69 +
 .../org/apache/storm/jdbc/spout/UserSpout.java  |   90 +
 .../jdbc/topology/AbstractUserTopology.java     |  106 +
 .../jdbc/topology/UserPersistanceTopology.java  |   48 +
 .../UserPersistanceTridentTopology.java         |   61 +
 external/storm-jdbc/src/test/sql/test.sql       |    1 +
 .../src/jvm/storm/kafka/PartitionManager.java   |    5 +-
 external/storm-redis/LICENSE                    |  202 +
 external/storm-redis/README.md                  |  137 +
 external/storm-redis/pom.xml                    |   65 +
 .../storm/redis/bolt/AbstractRedisBolt.java     |   67 +
 .../trident/mapper/TridentTupleMapper.java      |   27 +
 .../trident/state/RedisClusterMapState.java     |  294 +
 .../redis/trident/state/RedisClusterState.java  |   80 +
 .../trident/state/RedisClusterStateQuerier.java |   78 +
 .../trident/state/RedisClusterStateUpdater.java |   76 +
 .../redis/trident/state/RedisMapState.java      |  323 +
 .../storm/redis/trident/state/RedisState.java   |   83 +
 .../redis/trident/state/RedisStateQuerier.java  |   70 +
 .../state/RedisStateSetCountQuerier.java        |   74 +
 .../trident/state/RedisStateSetUpdater.java     |   80 +
 .../redis/trident/state/RedisStateUpdater.java  |   75 +
 .../redis/util/config/JedisClusterConfig.java   |   82 +
 .../redis/util/config/JedisPoolConfig.java      |   97 +
 .../util/container/JedisClusterContainer.java   |   47 +
 .../JedisCommandsContainerBuilder.java          |   38 +
 .../JedisCommandsInstanceContainer.java         |   25 +
 .../redis/util/container/JedisContainer.java    |   65 +
 .../storm/redis/topology/LookupWordCount.java   |  127 +
 .../redis/topology/PersistentWordCount.java     |  117 +
 .../storm/redis/topology/WordCounter.java       |   58 +
 .../apache/storm/redis/topology/WordSpout.java  |   88 +
 .../storm/redis/trident/PrintFunction.java      |   40 +
 .../redis/trident/WordCountTridentRedis.java    |   97 +
 .../trident/WordCountTridentRedisCluster.java   |  103 +
 .../WordCountTridentRedisClusterMap.java        |  101 +
 .../redis/trident/WordCountTridentRedisMap.java |   95 +
 .../redis/trident/WordCountTupleMapper.java     |   16 +
 pom.xml                                         |   16 +-
 storm-core/pom.xml                              |   13 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj |   64 -
 storm-core/src/clj/backtype/storm/clojure.clj   |    2 +-
 storm-core/src/clj/backtype/storm/config.clj    |    3 +-
 .../src/clj/backtype/storm/daemon/drpc.clj      |    6 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |   30 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   29 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  191 +-
 .../src/clj/backtype/storm/daemon/task.clj      |   20 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |   70 +-
 .../src/clj/backtype/storm/messaging/local.clj  |    2 +-
 storm-core/src/clj/backtype/storm/tuple.clj     |    4 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |   63 +-
 .../src/clj/backtype/storm/ui/helpers.clj       |    1 +
 storm-core/src/clj/backtype/storm/util.clj      |    3 +
 .../storm/drpc/DRPCInvocationsClient.java       |    6 +
 .../src/jvm/backtype/storm/drpc/DRPCSpout.java  |   14 +-
 .../jvm/backtype/storm/drpc/ReturnResults.java  |    8 +-
 .../storm/generated/AlreadyAliveException.java  |  149 +-
 .../storm/generated/AuthorizationException.java |  149 +-
 .../src/jvm/backtype/storm/generated/Bolt.java  |  194 +-
 .../jvm/backtype/storm/generated/BoltStats.java |  908 +-
 .../storm/generated/ClusterSummary.java         |  339 +-
 .../storm/generated/ComponentCommon.java        |  426 +-
 .../storm/generated/ComponentObject.java        |   86 +-
 .../backtype/storm/generated/Credentials.java   |  220 +-
 .../storm/generated/DRPCExecutionException.java |  149 +-
 .../backtype/storm/generated/DRPCRequest.java   |  185 +-
 .../storm/generated/DistributedRPC.java         |  529 +-
 .../generated/DistributedRPCInvocations.java    | 1199 ++-
 .../jvm/backtype/storm/generated/ErrorInfo.java |  300 +-
 .../backtype/storm/generated/ExecutorInfo.java  |  198 +-
 .../storm/generated/ExecutorSpecificStats.java  |   72 +-
 .../backtype/storm/generated/ExecutorStats.java |  609 +-
 .../storm/generated/ExecutorSummary.java        |  371 +-
 .../storm/generated/GetInfoOptions.java         |  166 +-
 .../storm/generated/GlobalStreamId.java         |  185 +-
 .../jvm/backtype/storm/generated/Grouping.java  |  163 +-
 .../generated/InvalidTopologyException.java     |  149 +-
 .../backtype/storm/generated/JavaObject.java    |  239 +-
 .../backtype/storm/generated/JavaObjectArg.java |  108 +-
 .../backtype/storm/generated/KillOptions.java   |  176 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 9177 +++++++++++++-----
 .../storm/generated/NotAliveException.java      |  149 +-
 .../backtype/storm/generated/NullStruct.java    |  112 +-
 .../storm/generated/NumErrorsChoice.java        |    3 +-
 .../storm/generated/RebalanceOptions.java       |  348 +-
 .../storm/generated/ShellComponent.java         |  202 +-
 .../jvm/backtype/storm/generated/SpoutSpec.java |  194 +-
 .../backtype/storm/generated/SpoutStats.java    |  614 +-
 .../storm/generated/StateSpoutSpec.java         |  194 +-
 .../backtype/storm/generated/StormTopology.java |  410 +-
 .../backtype/storm/generated/StreamInfo.java    |  249 +-
 .../backtype/storm/generated/SubmitOptions.java |  208 +-
 .../storm/generated/SupervisorSummary.java      |  309 +-
 .../backtype/storm/generated/TopologyInfo.java  |  609 +-
 .../storm/generated/TopologyInitialStatus.java  |    3 +-
 .../storm/generated/TopologySummary.java        |  486 +-
 .../storm/messaging/ConnectionWithStatus.java   |   32 +
 .../backtype/storm/messaging/netty/Client.java  |  712 +-
 .../messaging/netty/SaslStormClientHandler.java |    5 +-
 .../backtype/storm/messaging/netty/Server.java  |  182 +-
 .../netty/StormClientPipelineFactory.java       |    5 +-
 .../security/auth/SimpleTransportPlugin.java    |    2 +-
 .../trident/drpc/ReturnResultsReducer.java      |    4 +-
 .../trident/spout/RichSpoutBatchExecutor.java   |    1 +
 storm-core/src/py/storm/DistributedRPC-remote   |   35 +-
 storm-core/src/py/storm/DistributedRPC.py       |   38 +-
 .../py/storm/DistributedRPCInvocations-remote   |   43 +-
 .../src/py/storm/DistributedRPCInvocations.py   |   95 +-
 storm-core/src/py/storm/Nimbus-remote           |  111 +-
 storm-core/src/py/storm/Nimbus.py               |  640 +-
 storm-core/src/py/storm/constants.py            |    6 +-
 storm-core/src/py/storm/ttypes.py               | 1290 +--
 storm-core/src/ui/public/index.html             |    1 +
 .../test/clj/backtype/storm/clojure_test.clj    |    9 +-
 .../test/clj/backtype/storm/drpc_test.clj       |    8 +-
 .../test/clj/backtype/storm/grouping_test.clj   |    9 +-
 .../clj/backtype/storm/integration_test.clj     |   10 +-
 .../storm/messaging/netty_integration_test.clj  |    5 +-
 .../storm/messaging/netty_unit_test.clj         |   73 +-
 .../test/clj/backtype/storm/messaging_test.clj  |    6 +-
 .../test/clj/backtype/storm/metrics_test.clj    |    8 +-
 .../test/clj/backtype/storm/multilang_test.clj  |    6 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |   21 +-
 .../scheduler/multitenant_scheduler_test.clj    |    4 +-
 .../test/clj/backtype/storm/scheduler_test.clj  |    4 +-
 .../storm/security/auth/AuthUtils_test.clj      |    6 +-
 .../backtype/storm/security/auth/auth_test.clj  |   13 +-
 .../storm/security/auth/drpc_auth_test.clj      |    7 +-
 .../storm/security/auth/nimbus_auth_test.clj    |   10 +-
 .../clj/backtype/storm/subtopology_test.clj     |    9 +-
 .../test/clj/backtype/storm/supervisor_test.clj |   16 +-
 .../test/clj/backtype/storm/tick_tuple_test.clj |    7 +-
 .../clj/backtype/storm/transactional_test.clj   |   14 +-
 .../test/clj/backtype/storm/worker_test.clj     |   37 +
 storm-dist/binary/src/main/assembly/binary.xml  |   38 +-
 203 files changed, 32972 insertions(+), 8388 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/storm/blob/5b67b571/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/config.clj
index 00396c7,483c6ff..5e7893b
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@@ -212,8 -211,7 +212,7 @@@
    (let [stormroot (supervisor-stormdist-root conf storm-id)
          conf-path (supervisor-stormconf-path stormroot)
          topology-path (supervisor-stormcode-path stormroot)]
-     (merge conf (Utils/javaDeserialize (FileUtils/readFileToByteArray (File. conf-path)) java.util.Map))
-     ))
 -    (merge conf (clojurify-structure (Utils/deserialize (FileUtils/readFileToByteArray (File. conf-path)))))))
++    (merge conf (clojurify-structure (Utils/javaDeserialize (FileUtils/readFileToByteArray (File. conf-path)))))))
  
  (defn read-supervisor-topology
    [conf storm-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/5b67b571/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/executor.clj
index a1ecc4a,ec68d45..0e083ea
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@@ -15,23 -15,29 +15,31 @@@
  ;; limitations under the License.
  (ns backtype.storm.daemon.executor
    (:use [backtype.storm.daemon common])
-   (:use [backtype.storm bootstrap])
-   (:import [backtype.storm ICredentialsListener]
-            [backtype.storm.generated Grouping]
++  (:import [backtype.storm.generated Grouping]
 +           [java.io Serializable])
+   (:use [backtype.storm util config log timer stats])
+   (:import [java.util List Random HashMap ArrayList LinkedList Map])
+   (:import [backtype.storm ICredentialsListener])
    (:import [backtype.storm.hooks ITaskHook])
-   (:import [backtype.storm.tuple Tuple])
-   (:import [backtype.storm.spout ISpoutWaitStrategy])
+   (:import [backtype.storm.tuple Tuple Fields TupleImpl MessageId])
+   (:import [backtype.storm.spout ISpoutWaitStrategy ISpout SpoutOutputCollector ISpoutOutputCollector])
    (:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo
              EmitInfo BoltFailInfo BoltAckInfo BoltExecuteInfo])
-   (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint])
-   (:import [backtype.storm Config])
+   (:import [backtype.storm.grouping CustomStreamGrouping])
+   (:import [backtype.storm.task WorkerTopologyContext IBolt OutputCollector IOutputCollector])
+   (:import [backtype.storm.generated GlobalStreamId])
+   (:import [backtype.storm.utils Utils MutableObject RotatingMap RotatingMap$ExpiredCallback MutableLong Time])
+   (:import [com.lmax.disruptor InsufficientCapacityException])
+   (:import [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer])
+   (:import [backtype.storm.daemon Shutdownable])
+   (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
+   (:import [backtype.storm Config Constants])
    (:import [java.util.concurrent ConcurrentLinkedQueue])
-   (:require [backtype.storm [tuple :as tuple]])
+   (:require [backtype.storm [tuple :as tuple] [thrift :as thrift]
+              [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
    (:require [backtype.storm.daemon [task :as task]])
-   (:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics]))
- 
- (bootstrap)
+   (:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics])
+   (:require [clojure.set :as set]))
  
  (defn- mk-fields-grouper [^Fields out-fields ^Fields group-fields ^List target-tasks]
    (let [num-tasks (count target-tasks)

http://git-wip-us.apache.org/repos/asf/storm/blob/5b67b571/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index d79b542,f1b4082..2c7b82c
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@@ -14,10 -14,14 +14,14 @@@
  ;; See the License for the specific language governing permissions and
  ;; limitations under the License.
  (ns backtype.storm.daemon.nimbus
-   (:import [java.nio ByteBuffer]
-            [java.util Collections]
-            [backtype.storm.generated StormTopology])
-   (:import [java.io FileNotFoundException])
+   (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
+   (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
+   (:import [org.apache.thrift.exception])
+   (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
+   (:import [org.apache.commons.io FileUtils])
 -  (:import [java.nio ByteBuffer]
++  (:import [java.nio ByteBuffer])
+            [java.util Collections HashMap])
+   (:import [java.io FileNotFoundException File FileOutputStream])
    (:import [java.nio.channels Channels WritableByteChannel])
    (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils])
    (:use [backtype.storm.scheduler.DefaultScheduler])

http://git-wip-us.apache.org/repos/asf/storm/blob/5b67b571/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/5b67b571/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/nimbus_test.clj
index 7671f58,99d1e32..f9502c2
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@@ -15,18 -15,25 +15,25 @@@
  ;; limitations under the License.
  (ns backtype.storm.nimbus-test
    (:use [clojure test])
 -  (:require [backtype.storm [util :as util]])
 +  (:require [backtype.storm [util :as util] [stats :as stats]])
    (:require [backtype.storm.daemon [nimbus :as nimbus]])
-   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
+   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount
+             TestAggregatesCounter TestPlannerSpout TestPlannerBolt])
    (:import [backtype.storm.scheduler INimbus])
-   (:import [backtype.storm.generated Credentials])
-   (:use [backtype.storm bootstrap testing MockAutoCred])
+   (:import [backtype.storm.generated Credentials NotAliveException SubmitOptions
+             TopologyInitialStatus AlreadyAliveException KillOptions RebalanceOptions
+             InvalidTopologyException AuthorizationException])
+   (:import [java.util HashMap])
+   (:import [java.io File])
+   (:import [backtype.storm.utils Time])
+   (:import [org.apache.commons.io FileUtils])
+   (:use [backtype.storm testing MockAutoCred util config log timer])
    (:use [backtype.storm.daemon common])
    (:require [conjure.core])
-   (:use [conjure core])
-   )
- 
- (bootstrap)
+   (:require [backtype.storm
+              [thrift :as thrift]
+              [cluster :as cluster]])
+   (:use [conjure core]))
  
  (defn storm-component->task-info [cluster storm-name]
    (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)


[02/16] storm git commit: STORM-634: Converting SupervisorInfo, Assignment, StormBase, TopologyStatus, ZKWorkerHeartbeat, ErrorInfo, Credentials to thrift and defaulting the serialization delegate to thrift serialization. Added class as a param to serializatio

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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
new file mode 100644
index 0000000..523b1c0
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
@@ -0,0 +1,1030 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo");
+
+  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ASSIGNMENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("assignment_id", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField USED_PORTS_FIELD_DESC = new org.apache.thrift.protocol.TField("used_ports", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField META_FIELD_DESC = new org.apache.thrift.protocol.TField("meta", org.apache.thrift.protocol.TType.LIST, (short)5);
+  private static final org.apache.thrift.protocol.TField SCHEDULER_META_FIELD_DESC = new org.apache.thrift.protocol.TField("scheduler_meta", org.apache.thrift.protocol.TType.MAP, (short)6);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I64, (short)7);
+
+  private long time_secs; // required
+  private String hostname; // required
+  private String assignment_id; // required
+  private List<Long> used_ports; // required
+  private List<Long> meta; // required
+  private Map<String,String> scheduler_meta; // required
+  private long uptime_secs; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TIME_SECS((short)1, "time_secs"),
+    HOSTNAME((short)2, "hostname"),
+    ASSIGNMENT_ID((short)3, "assignment_id"),
+    USED_PORTS((short)4, "used_ports"),
+    META((short)5, "meta"),
+    SCHEDULER_META((short)6, "scheduler_meta"),
+    UPTIME_SECS((short)7, "uptime_secs");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TIME_SECS
+          return TIME_SECS;
+        case 2: // HOSTNAME
+          return HOSTNAME;
+        case 3: // ASSIGNMENT_ID
+          return ASSIGNMENT_ID;
+        case 4: // USED_PORTS
+          return USED_PORTS;
+        case 5: // META
+          return META;
+        case 6: // SCHEDULER_META
+          return SCHEDULER_META;
+        case 7: // UPTIME_SECS
+          return UPTIME_SECS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_SECS_ISSET_ID = 0;
+  private static final int __UPTIME_SECS_ISSET_ID = 1;
+  private BitSet __isset_bit_vector = new BitSet(2);
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ASSIGNMENT_ID, new org.apache.thrift.meta_data.FieldMetaData("assignment_id", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.USED_PORTS, new org.apache.thrift.meta_data.FieldMetaData("used_ports", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.META, new org.apache.thrift.meta_data.FieldMetaData("meta", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.SCHEDULER_META, new org.apache.thrift.meta_data.FieldMetaData("scheduler_meta", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorInfo.class, metaDataMap);
+  }
+
+  public SupervisorInfo() {
+  }
+
+  public SupervisorInfo(
+    long time_secs,
+    String hostname)
+  {
+    this();
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+    this.hostname = hostname;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SupervisorInfo(SupervisorInfo other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    this.time_secs = other.time_secs;
+    if (other.is_set_hostname()) {
+      this.hostname = other.hostname;
+    }
+    if (other.is_set_assignment_id()) {
+      this.assignment_id = other.assignment_id;
+    }
+    if (other.is_set_used_ports()) {
+      List<Long> __this__used_ports = new ArrayList<Long>();
+      for (Long other_element : other.used_ports) {
+        __this__used_ports.add(other_element);
+      }
+      this.used_ports = __this__used_ports;
+    }
+    if (other.is_set_meta()) {
+      List<Long> __this__meta = new ArrayList<Long>();
+      for (Long other_element : other.meta) {
+        __this__meta.add(other_element);
+      }
+      this.meta = __this__meta;
+    }
+    if (other.is_set_scheduler_meta()) {
+      Map<String,String> __this__scheduler_meta = new HashMap<String,String>();
+      for (Map.Entry<String, String> other_element : other.scheduler_meta.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        String other_element_value = other_element.getValue();
+
+        String __this__scheduler_meta_copy_key = other_element_key;
+
+        String __this__scheduler_meta_copy_value = other_element_value;
+
+        __this__scheduler_meta.put(__this__scheduler_meta_copy_key, __this__scheduler_meta_copy_value);
+      }
+      this.scheduler_meta = __this__scheduler_meta;
+    }
+    this.uptime_secs = other.uptime_secs;
+  }
+
+  public SupervisorInfo deepCopy() {
+    return new SupervisorInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    set_time_secs_isSet(false);
+    this.time_secs = 0;
+    this.hostname = null;
+    this.assignment_id = null;
+    this.used_ports = null;
+    this.meta = null;
+    this.scheduler_meta = null;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+  }
+
+  public long get_time_secs() {
+    return this.time_secs;
+  }
+
+  public void set_time_secs(long time_secs) {
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  public void unset_time_secs() {
+    __isset_bit_vector.clear(__TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_secs() {
+    return __isset_bit_vector.get(__TIME_SECS_ISSET_ID);
+  }
+
+  public void set_time_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__TIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_hostname() {
+    return this.hostname;
+  }
+
+  public void set_hostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  public void unset_hostname() {
+    this.hostname = null;
+  }
+
+  /** Returns true if field hostname is set (has been assigned a value) and false otherwise */
+  public boolean is_set_hostname() {
+    return this.hostname != null;
+  }
+
+  public void set_hostname_isSet(boolean value) {
+    if (!value) {
+      this.hostname = null;
+    }
+  }
+
+  public String get_assignment_id() {
+    return this.assignment_id;
+  }
+
+  public void set_assignment_id(String assignment_id) {
+    this.assignment_id = assignment_id;
+  }
+
+  public void unset_assignment_id() {
+    this.assignment_id = null;
+  }
+
+  /** Returns true if field assignment_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assignment_id() {
+    return this.assignment_id != null;
+  }
+
+  public void set_assignment_id_isSet(boolean value) {
+    if (!value) {
+      this.assignment_id = null;
+    }
+  }
+
+  public int get_used_ports_size() {
+    return (this.used_ports == null) ? 0 : this.used_ports.size();
+  }
+
+  public java.util.Iterator<Long> get_used_ports_iterator() {
+    return (this.used_ports == null) ? null : this.used_ports.iterator();
+  }
+
+  public void add_to_used_ports(long elem) {
+    if (this.used_ports == null) {
+      this.used_ports = new ArrayList<Long>();
+    }
+    this.used_ports.add(elem);
+  }
+
+  public List<Long> get_used_ports() {
+    return this.used_ports;
+  }
+
+  public void set_used_ports(List<Long> used_ports) {
+    this.used_ports = used_ports;
+  }
+
+  public void unset_used_ports() {
+    this.used_ports = null;
+  }
+
+  /** Returns true if field used_ports is set (has been assigned a value) and false otherwise */
+  public boolean is_set_used_ports() {
+    return this.used_ports != null;
+  }
+
+  public void set_used_ports_isSet(boolean value) {
+    if (!value) {
+      this.used_ports = null;
+    }
+  }
+
+  public int get_meta_size() {
+    return (this.meta == null) ? 0 : this.meta.size();
+  }
+
+  public java.util.Iterator<Long> get_meta_iterator() {
+    return (this.meta == null) ? null : this.meta.iterator();
+  }
+
+  public void add_to_meta(long elem) {
+    if (this.meta == null) {
+      this.meta = new ArrayList<Long>();
+    }
+    this.meta.add(elem);
+  }
+
+  public List<Long> get_meta() {
+    return this.meta;
+  }
+
+  public void set_meta(List<Long> meta) {
+    this.meta = meta;
+  }
+
+  public void unset_meta() {
+    this.meta = null;
+  }
+
+  /** Returns true if field meta is set (has been assigned a value) and false otherwise */
+  public boolean is_set_meta() {
+    return this.meta != null;
+  }
+
+  public void set_meta_isSet(boolean value) {
+    if (!value) {
+      this.meta = null;
+    }
+  }
+
+  public int get_scheduler_meta_size() {
+    return (this.scheduler_meta == null) ? 0 : this.scheduler_meta.size();
+  }
+
+  public void put_to_scheduler_meta(String key, String val) {
+    if (this.scheduler_meta == null) {
+      this.scheduler_meta = new HashMap<String,String>();
+    }
+    this.scheduler_meta.put(key, val);
+  }
+
+  public Map<String,String> get_scheduler_meta() {
+    return this.scheduler_meta;
+  }
+
+  public void set_scheduler_meta(Map<String,String> scheduler_meta) {
+    this.scheduler_meta = scheduler_meta;
+  }
+
+  public void unset_scheduler_meta() {
+    this.scheduler_meta = null;
+  }
+
+  /** Returns true if field scheduler_meta is set (has been assigned a value) and false otherwise */
+  public boolean is_set_scheduler_meta() {
+    return this.scheduler_meta != null;
+  }
+
+  public void set_scheduler_meta_isSet(boolean value) {
+    if (!value) {
+      this.scheduler_meta = null;
+    }
+  }
+
+  public long get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(long uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TIME_SECS:
+      if (value == null) {
+        unset_time_secs();
+      } else {
+        set_time_secs((Long)value);
+      }
+      break;
+
+    case HOSTNAME:
+      if (value == null) {
+        unset_hostname();
+      } else {
+        set_hostname((String)value);
+      }
+      break;
+
+    case ASSIGNMENT_ID:
+      if (value == null) {
+        unset_assignment_id();
+      } else {
+        set_assignment_id((String)value);
+      }
+      break;
+
+    case USED_PORTS:
+      if (value == null) {
+        unset_used_ports();
+      } else {
+        set_used_ports((List<Long>)value);
+      }
+      break;
+
+    case META:
+      if (value == null) {
+        unset_meta();
+      } else {
+        set_meta((List<Long>)value);
+      }
+      break;
+
+    case SCHEDULER_META:
+      if (value == null) {
+        unset_scheduler_meta();
+      } else {
+        set_scheduler_meta((Map<String,String>)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TIME_SECS:
+      return Long.valueOf(get_time_secs());
+
+    case HOSTNAME:
+      return get_hostname();
+
+    case ASSIGNMENT_ID:
+      return get_assignment_id();
+
+    case USED_PORTS:
+      return get_used_ports();
+
+    case META:
+      return get_meta();
+
+    case SCHEDULER_META:
+      return get_scheduler_meta();
+
+    case UPTIME_SECS:
+      return Long.valueOf(get_uptime_secs());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TIME_SECS:
+      return is_set_time_secs();
+    case HOSTNAME:
+      return is_set_hostname();
+    case ASSIGNMENT_ID:
+      return is_set_assignment_id();
+    case USED_PORTS:
+      return is_set_used_ports();
+    case META:
+      return is_set_meta();
+    case SCHEDULER_META:
+      return is_set_scheduler_meta();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SupervisorInfo)
+      return this.equals((SupervisorInfo)that);
+    return false;
+  }
+
+  public boolean equals(SupervisorInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_time_secs = true;
+    boolean that_present_time_secs = true;
+    if (this_present_time_secs || that_present_time_secs) {
+      if (!(this_present_time_secs && that_present_time_secs))
+        return false;
+      if (this.time_secs != that.time_secs)
+        return false;
+    }
+
+    boolean this_present_hostname = true && this.is_set_hostname();
+    boolean that_present_hostname = true && that.is_set_hostname();
+    if (this_present_hostname || that_present_hostname) {
+      if (!(this_present_hostname && that_present_hostname))
+        return false;
+      if (!this.hostname.equals(that.hostname))
+        return false;
+    }
+
+    boolean this_present_assignment_id = true && this.is_set_assignment_id();
+    boolean that_present_assignment_id = true && that.is_set_assignment_id();
+    if (this_present_assignment_id || that_present_assignment_id) {
+      if (!(this_present_assignment_id && that_present_assignment_id))
+        return false;
+      if (!this.assignment_id.equals(that.assignment_id))
+        return false;
+    }
+
+    boolean this_present_used_ports = true && this.is_set_used_ports();
+    boolean that_present_used_ports = true && that.is_set_used_ports();
+    if (this_present_used_ports || that_present_used_ports) {
+      if (!(this_present_used_ports && that_present_used_ports))
+        return false;
+      if (!this.used_ports.equals(that.used_ports))
+        return false;
+    }
+
+    boolean this_present_meta = true && this.is_set_meta();
+    boolean that_present_meta = true && that.is_set_meta();
+    if (this_present_meta || that_present_meta) {
+      if (!(this_present_meta && that_present_meta))
+        return false;
+      if (!this.meta.equals(that.meta))
+        return false;
+    }
+
+    boolean this_present_scheduler_meta = true && this.is_set_scheduler_meta();
+    boolean that_present_scheduler_meta = true && that.is_set_scheduler_meta();
+    if (this_present_scheduler_meta || that_present_scheduler_meta) {
+      if (!(this_present_scheduler_meta && that_present_scheduler_meta))
+        return false;
+      if (!this.scheduler_meta.equals(that.scheduler_meta))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true && this.is_set_uptime_secs();
+    boolean that_present_uptime_secs = true && that.is_set_uptime_secs();
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_time_secs = true;
+    builder.append(present_time_secs);
+    if (present_time_secs)
+      builder.append(time_secs);
+
+    boolean present_hostname = true && (is_set_hostname());
+    builder.append(present_hostname);
+    if (present_hostname)
+      builder.append(hostname);
+
+    boolean present_assignment_id = true && (is_set_assignment_id());
+    builder.append(present_assignment_id);
+    if (present_assignment_id)
+      builder.append(assignment_id);
+
+    boolean present_used_ports = true && (is_set_used_ports());
+    builder.append(present_used_ports);
+    if (present_used_ports)
+      builder.append(used_ports);
+
+    boolean present_meta = true && (is_set_meta());
+    builder.append(present_meta);
+    if (present_meta)
+      builder.append(meta);
+
+    boolean present_scheduler_meta = true && (is_set_scheduler_meta());
+    builder.append(present_scheduler_meta);
+    if (present_scheduler_meta)
+      builder.append(scheduler_meta);
+
+    boolean present_uptime_secs = true && (is_set_uptime_secs());
+    builder.append(present_uptime_secs);
+    if (present_uptime_secs)
+      builder.append(uptime_secs);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(SupervisorInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    SupervisorInfo typedOther = (SupervisorInfo)other;
+
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(typedOther.is_set_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, typedOther.time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_hostname()).compareTo(typedOther.is_set_hostname());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_hostname()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, typedOther.hostname);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assignment_id()).compareTo(typedOther.is_set_assignment_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assignment_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assignment_id, typedOther.assignment_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_used_ports()).compareTo(typedOther.is_set_used_ports());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_used_ports()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_ports, typedOther.used_ports);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_meta()).compareTo(typedOther.is_set_meta());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_meta()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.meta, typedOther.meta);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_scheduler_meta()).compareTo(typedOther.is_set_scheduler_meta());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_scheduler_meta()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scheduler_meta, typedOther.scheduler_meta);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        break;
+      }
+      switch (field.id) {
+        case 1: // TIME_SECS
+          if (field.type == org.apache.thrift.protocol.TType.I64) {
+            this.time_secs = iprot.readI64();
+            set_time_secs_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // HOSTNAME
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.hostname = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // ASSIGNMENT_ID
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.assignment_id = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // USED_PORTS
+          if (field.type == org.apache.thrift.protocol.TType.LIST) {
+            {
+              org.apache.thrift.protocol.TList _list168 = iprot.readListBegin();
+              this.used_ports = new ArrayList<Long>(_list168.size);
+              for (int _i169 = 0; _i169 < _list168.size; ++_i169)
+              {
+                long _elem170; // required
+                _elem170 = iprot.readI64();
+                this.used_ports.add(_elem170);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 5: // META
+          if (field.type == org.apache.thrift.protocol.TType.LIST) {
+            {
+              org.apache.thrift.protocol.TList _list171 = iprot.readListBegin();
+              this.meta = new ArrayList<Long>(_list171.size);
+              for (int _i172 = 0; _i172 < _list171.size; ++_i172)
+              {
+                long _elem173; // required
+                _elem173 = iprot.readI64();
+                this.meta.add(_elem173);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 6: // SCHEDULER_META
+          if (field.type == org.apache.thrift.protocol.TType.MAP) {
+            {
+              org.apache.thrift.protocol.TMap _map174 = iprot.readMapBegin();
+              this.scheduler_meta = new HashMap<String,String>(2*_map174.size);
+              for (int _i175 = 0; _i175 < _map174.size; ++_i175)
+              {
+                String _key176; // required
+                String _val177; // required
+                _key176 = iprot.readString();
+                _val177 = iprot.readString();
+                this.scheduler_meta.put(_key176, _val177);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 7: // UPTIME_SECS
+          if (field.type == org.apache.thrift.protocol.TType.I64) {
+            this.uptime_secs = iprot.readI64();
+            set_uptime_secs_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+    oprot.writeI64(this.time_secs);
+    oprot.writeFieldEnd();
+    if (this.hostname != null) {
+      oprot.writeFieldBegin(HOSTNAME_FIELD_DESC);
+      oprot.writeString(this.hostname);
+      oprot.writeFieldEnd();
+    }
+    if (this.assignment_id != null) {
+      if (is_set_assignment_id()) {
+        oprot.writeFieldBegin(ASSIGNMENT_ID_FIELD_DESC);
+        oprot.writeString(this.assignment_id);
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.used_ports != null) {
+      if (is_set_used_ports()) {
+        oprot.writeFieldBegin(USED_PORTS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, this.used_ports.size()));
+          for (long _iter178 : this.used_ports)
+          {
+            oprot.writeI64(_iter178);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.meta != null) {
+      if (is_set_meta()) {
+        oprot.writeFieldBegin(META_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, this.meta.size()));
+          for (long _iter179 : this.meta)
+          {
+            oprot.writeI64(_iter179);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.scheduler_meta != null) {
+      if (is_set_scheduler_meta()) {
+        oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, this.scheduler_meta.size()));
+          for (Map.Entry<String, String> _iter180 : this.scheduler_meta.entrySet())
+          {
+            oprot.writeString(_iter180.getKey());
+            oprot.writeString(_iter180.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+    }
+    if (is_set_uptime_secs()) {
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI64(this.uptime_secs);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SupervisorInfo(");
+    boolean first = true;
+
+    sb.append("time_secs:");
+    sb.append(this.time_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("hostname:");
+    if (this.hostname == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.hostname);
+    }
+    first = false;
+    if (is_set_assignment_id()) {
+      if (!first) sb.append(", ");
+      sb.append("assignment_id:");
+      if (this.assignment_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.assignment_id);
+      }
+      first = false;
+    }
+    if (is_set_used_ports()) {
+      if (!first) sb.append(", ");
+      sb.append("used_ports:");
+      if (this.used_ports == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.used_ports);
+      }
+      first = false;
+    }
+    if (is_set_meta()) {
+      if (!first) sb.append(", ");
+      sb.append("meta:");
+      if (this.meta == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.meta);
+      }
+      first = false;
+    }
+    if (is_set_scheduler_meta()) {
+      if (!first) sb.append(", ");
+      sb.append("scheduler_meta:");
+      if (this.scheduler_meta == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.scheduler_meta);
+      }
+      first = false;
+    }
+    if (is_set_uptime_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("uptime_secs:");
+      sb.append(this.uptime_secs);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_time_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_hostname()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString());
+    }
+
+  }
+
+  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_bit_vector = new BitSet(1);
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java b/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
new file mode 100644
index 0000000..a3c65cf
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
@@ -0,0 +1,335 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TopologyActionOptions extends org.apache.thrift.TUnion<TopologyActionOptions, TopologyActionOptions._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyActionOptions");
+  private static final org.apache.thrift.protocol.TField KILL_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("kill_options", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField REBALANCE_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("rebalance_options", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    KILL_OPTIONS((short)1, "kill_options"),
+    REBALANCE_OPTIONS((short)2, "rebalance_options");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // KILL_OPTIONS
+          return KILL_OPTIONS;
+        case 2: // REBALANCE_OPTIONS
+          return REBALANCE_OPTIONS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.KILL_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("kill_options", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KillOptions.class)));
+    tmpMap.put(_Fields.REBALANCE_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("rebalance_options", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RebalanceOptions.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyActionOptions.class, metaDataMap);
+  }
+
+  public TopologyActionOptions() {
+    super();
+  }
+
+  public TopologyActionOptions(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public TopologyActionOptions(TopologyActionOptions other) {
+    super(other);
+  }
+  public TopologyActionOptions deepCopy() {
+    return new TopologyActionOptions(this);
+  }
+
+  public static TopologyActionOptions kill_options(KillOptions value) {
+    TopologyActionOptions x = new TopologyActionOptions();
+    x.set_kill_options(value);
+    return x;
+  }
+
+  public static TopologyActionOptions rebalance_options(RebalanceOptions value) {
+    TopologyActionOptions x = new TopologyActionOptions();
+    x.set_rebalance_options(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case KILL_OPTIONS:
+        if (value instanceof KillOptions) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type KillOptions for field 'kill_options', but got " + value.getClass().getSimpleName());
+      case REBALANCE_OPTIONS:
+        if (value instanceof RebalanceOptions) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type RebalanceOptions for field 'rebalance_options', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object readValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case KILL_OPTIONS:
+          if (field.type == KILL_OPTIONS_FIELD_DESC.type) {
+            KillOptions kill_options;
+            kill_options = new KillOptions();
+            kill_options.read(iprot);
+            return kill_options;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case REBALANCE_OPTIONS:
+          if (field.type == REBALANCE_OPTIONS_FIELD_DESC.type) {
+            RebalanceOptions rebalance_options;
+            rebalance_options = new RebalanceOptions();
+            rebalance_options.read(iprot);
+            return rebalance_options;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void writeValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case KILL_OPTIONS:
+        KillOptions kill_options = (KillOptions)value_;
+        kill_options.write(oprot);
+        return;
+      case REBALANCE_OPTIONS:
+        RebalanceOptions rebalance_options = (RebalanceOptions)value_;
+        rebalance_options.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case KILL_OPTIONS:
+        return KILL_OPTIONS_FIELD_DESC;
+      case REBALANCE_OPTIONS:
+        return REBALANCE_OPTIONS_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public KillOptions get_kill_options() {
+    if (getSetField() == _Fields.KILL_OPTIONS) {
+      return (KillOptions)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'kill_options' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_kill_options(KillOptions value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.KILL_OPTIONS;
+    value_ = value;
+  }
+
+  public RebalanceOptions get_rebalance_options() {
+    if (getSetField() == _Fields.REBALANCE_OPTIONS) {
+      return (RebalanceOptions)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'rebalance_options' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_rebalance_options(RebalanceOptions value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.REBALANCE_OPTIONS;
+    value_ = value;
+  }
+
+  public boolean is_set_kill_options() {
+    return setField_ == _Fields.KILL_OPTIONS;
+  }
+
+
+  public boolean is_set_rebalance_options() {
+    return setField_ == _Fields.REBALANCE_OPTIONS;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof TopologyActionOptions) {
+      return equals((TopologyActionOptions)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(TopologyActionOptions other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(TopologyActionOptions other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder hcb = new HashCodeBuilder();
+    hcb.append(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      hcb.append(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        hcb.append(value);
+      }
+    }
+    return hcb.toHashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}

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

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
new file mode 100644
index 0000000..47d8428
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
@@ -0,0 +1,586 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartbeat, ZKWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ZKWorkerHeartbeat");
+
+  private static final org.apache.thrift.protocol.TField STORM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storm_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_stats", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+
+  private String storm_id; // required
+  private Map<ExecutorInfo,ExecutorStats> executor_stats; // required
+  private int time_secs; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    STORM_ID((short)1, "storm_id"),
+    EXECUTOR_STATS((short)2, "executor_stats"),
+    TIME_SECS((short)3, "time_secs");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STORM_ID
+          return STORM_ID;
+        case 2: // EXECUTOR_STATS
+          return EXECUTOR_STATS;
+        case 3: // TIME_SECS
+          return TIME_SECS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_SECS_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STORM_ID, new org.apache.thrift.meta_data.FieldMetaData("storm_id", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EXECUTOR_STATS, new org.apache.thrift.meta_data.FieldMetaData("executor_stats", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class),
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorStats.class))));
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ZKWorkerHeartbeat.class, metaDataMap);
+  }
+
+  public ZKWorkerHeartbeat() {
+  }
+
+  public ZKWorkerHeartbeat(
+    String storm_id,
+    Map<ExecutorInfo,ExecutorStats> executor_stats,
+    int time_secs)
+  {
+    this();
+    this.storm_id = storm_id;
+    this.executor_stats = executor_stats;
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ZKWorkerHeartbeat(ZKWorkerHeartbeat other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_storm_id()) {
+      this.storm_id = other.storm_id;
+    }
+    if (other.is_set_executor_stats()) {
+      Map<ExecutorInfo,ExecutorStats> __this__executor_stats = new HashMap<ExecutorInfo,ExecutorStats>();
+      for (Map.Entry<ExecutorInfo, ExecutorStats> other_element : other.executor_stats.entrySet()) {
+
+        ExecutorInfo other_element_key = other_element.getKey();
+        ExecutorStats other_element_value = other_element.getValue();
+
+        ExecutorInfo __this__executor_stats_copy_key = new ExecutorInfo(other_element_key);
+
+        ExecutorStats __this__executor_stats_copy_value = new ExecutorStats(other_element_value);
+
+        __this__executor_stats.put(__this__executor_stats_copy_key, __this__executor_stats_copy_value);
+      }
+      this.executor_stats = __this__executor_stats;
+    }
+    this.time_secs = other.time_secs;
+  }
+
+  public ZKWorkerHeartbeat deepCopy() {
+    return new ZKWorkerHeartbeat(this);
+  }
+
+  @Override
+  public void clear() {
+    this.storm_id = null;
+    this.executor_stats = null;
+    set_time_secs_isSet(false);
+    this.time_secs = 0;
+  }
+
+  public String get_storm_id() {
+    return this.storm_id;
+  }
+
+  public void set_storm_id(String storm_id) {
+    this.storm_id = storm_id;
+  }
+
+  public void unset_storm_id() {
+    this.storm_id = null;
+  }
+
+  /** Returns true if field storm_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_storm_id() {
+    return this.storm_id != null;
+  }
+
+  public void set_storm_id_isSet(boolean value) {
+    if (!value) {
+      this.storm_id = null;
+    }
+  }
+
+  public int get_executor_stats_size() {
+    return (this.executor_stats == null) ? 0 : this.executor_stats.size();
+  }
+
+  public void put_to_executor_stats(ExecutorInfo key, ExecutorStats val) {
+    if (this.executor_stats == null) {
+      this.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>();
+    }
+    this.executor_stats.put(key, val);
+  }
+
+  public Map<ExecutorInfo,ExecutorStats> get_executor_stats() {
+    return this.executor_stats;
+  }
+
+  public void set_executor_stats(Map<ExecutorInfo,ExecutorStats> executor_stats) {
+    this.executor_stats = executor_stats;
+  }
+
+  public void unset_executor_stats() {
+    this.executor_stats = null;
+  }
+
+  /** Returns true if field executor_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_stats() {
+    return this.executor_stats != null;
+  }
+
+  public void set_executor_stats_isSet(boolean value) {
+    if (!value) {
+      this.executor_stats = null;
+    }
+  }
+
+  public int get_time_secs() {
+    return this.time_secs;
+  }
+
+  public void set_time_secs(int time_secs) {
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  public void unset_time_secs() {
+    __isset_bit_vector.clear(__TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_secs() {
+    return __isset_bit_vector.get(__TIME_SECS_ISSET_ID);
+  }
+
+  public void set_time_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__TIME_SECS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STORM_ID:
+      if (value == null) {
+        unset_storm_id();
+      } else {
+        set_storm_id((String)value);
+      }
+      break;
+
+    case EXECUTOR_STATS:
+      if (value == null) {
+        unset_executor_stats();
+      } else {
+        set_executor_stats((Map<ExecutorInfo,ExecutorStats>)value);
+      }
+      break;
+
+    case TIME_SECS:
+      if (value == null) {
+        unset_time_secs();
+      } else {
+        set_time_secs((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STORM_ID:
+      return get_storm_id();
+
+    case EXECUTOR_STATS:
+      return get_executor_stats();
+
+    case TIME_SECS:
+      return Integer.valueOf(get_time_secs());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case STORM_ID:
+      return is_set_storm_id();
+    case EXECUTOR_STATS:
+      return is_set_executor_stats();
+    case TIME_SECS:
+      return is_set_time_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ZKWorkerHeartbeat)
+      return this.equals((ZKWorkerHeartbeat)that);
+    return false;
+  }
+
+  public boolean equals(ZKWorkerHeartbeat that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_storm_id = true && this.is_set_storm_id();
+    boolean that_present_storm_id = true && that.is_set_storm_id();
+    if (this_present_storm_id || that_present_storm_id) {
+      if (!(this_present_storm_id && that_present_storm_id))
+        return false;
+      if (!this.storm_id.equals(that.storm_id))
+        return false;
+    }
+
+    boolean this_present_executor_stats = true && this.is_set_executor_stats();
+    boolean that_present_executor_stats = true && that.is_set_executor_stats();
+    if (this_present_executor_stats || that_present_executor_stats) {
+      if (!(this_present_executor_stats && that_present_executor_stats))
+        return false;
+      if (!this.executor_stats.equals(that.executor_stats))
+        return false;
+    }
+
+    boolean this_present_time_secs = true;
+    boolean that_present_time_secs = true;
+    if (this_present_time_secs || that_present_time_secs) {
+      if (!(this_present_time_secs && that_present_time_secs))
+        return false;
+      if (this.time_secs != that.time_secs)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_storm_id = true && (is_set_storm_id());
+    builder.append(present_storm_id);
+    if (present_storm_id)
+      builder.append(storm_id);
+
+    boolean present_executor_stats = true && (is_set_executor_stats());
+    builder.append(present_executor_stats);
+    if (present_executor_stats)
+      builder.append(executor_stats);
+
+    boolean present_time_secs = true;
+    builder.append(present_time_secs);
+    if (present_time_secs)
+      builder.append(time_secs);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(ZKWorkerHeartbeat other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    ZKWorkerHeartbeat typedOther = (ZKWorkerHeartbeat)other;
+
+    lastComparison = Boolean.valueOf(is_set_storm_id()).compareTo(typedOther.is_set_storm_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_storm_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_id, typedOther.storm_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executor_stats()).compareTo(typedOther.is_set_executor_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_stats, typedOther.executor_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(typedOther.is_set_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, typedOther.time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        break;
+      }
+      switch (field.id) {
+        case 1: // STORM_ID
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.storm_id = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // EXECUTOR_STATS
+          if (field.type == org.apache.thrift.protocol.TType.MAP) {
+            {
+              org.apache.thrift.protocol.TMap _map213 = iprot.readMapBegin();
+              this.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map213.size);
+              for (int _i214 = 0; _i214 < _map213.size; ++_i214)
+              {
+                ExecutorInfo _key215; // required
+                ExecutorStats _val216; // required
+                _key215 = new ExecutorInfo();
+                _key215.read(iprot);
+                _val216 = new ExecutorStats();
+                _val216.read(iprot);
+                this.executor_stats.put(_key215, _val216);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // TIME_SECS
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.time_secs = iprot.readI32();
+            set_time_secs_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.storm_id != null) {
+      oprot.writeFieldBegin(STORM_ID_FIELD_DESC);
+      oprot.writeString(this.storm_id);
+      oprot.writeFieldEnd();
+    }
+    if (this.executor_stats != null) {
+      oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, this.executor_stats.size()));
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter217 : this.executor_stats.entrySet())
+        {
+          _iter217.getKey().write(oprot);
+          _iter217.getValue().write(oprot);
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.time_secs);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ZKWorkerHeartbeat(");
+    boolean first = true;
+
+    sb.append("storm_id:");
+    if (this.storm_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.storm_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executor_stats:");
+    if (this.executor_stats == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executor_stats);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("time_secs:");
+    sb.append(this.time_secs);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_storm_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'storm_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executor_stats()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executor_stats' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_time_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
+    }
+
+  }
+
+  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_bit_vector = new BitSet(1);
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
index b09bbe8..913baa6 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
@@ -17,11 +17,7 @@
  */
 package backtype.storm.serialization;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
+import java.io.*;
 import java.util.Map;
 
 public class DefaultSerializationDelegate implements SerializationDelegate {
@@ -45,13 +41,13 @@ public class DefaultSerializationDelegate implements SerializationDelegate {
     }
 
     @Override
-    public Object deserialize(byte[] bytes) {
+    public <T> T deserialize(byte[] bytes, Class<T> clazz) {
         try {
             ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
             ObjectInputStream ois = new ObjectInputStream(bis);
             Object ret = ois.readObject();
             ois.close();
-            return ret;
+            return (T)ret;
         } catch(IOException ioe) {
             throw new RuntimeException(ioe);
         } catch(ClassNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
index 6a7552f..1306ea9 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
@@ -42,11 +42,11 @@ public class GzipBridgeSerializationDelegate implements SerializationDelegate {
     }
 
     @Override
-    public Object deserialize(byte[] bytes) {
+    public <T> T deserialize(byte[] bytes, Class<T> clazz) {
         if (isGzipped(bytes)) {
-            return gzipDelegate.deserialize(bytes);
+            return gzipDelegate.deserialize(bytes, clazz);
         } else {
-            return defaultDelegate.deserialize(bytes);
+            return defaultDelegate.deserialize(bytes,clazz);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/serialization/GzipSerializationDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/GzipSerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/GzipSerializationDelegate.java
index e5a230f..3c8ee8b 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/GzipSerializationDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/GzipSerializationDelegate.java
@@ -17,11 +17,7 @@
  */
 package backtype.storm.serialization;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
+import java.io.*;
 import java.util.Map;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
@@ -51,14 +47,14 @@ public class GzipSerializationDelegate implements SerializationDelegate {
     }
 
     @Override
-    public Object deserialize(byte[] bytes) {
+    public <T> T deserialize(byte[] bytes, Class<T> clazz) {
         try {
             ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
             GZIPInputStream gis = new GZIPInputStream(bis);
             ObjectInputStream ois = new ObjectInputStream(gis);
             Object ret = ois.readObject();
             ois.close();
-            return ret;
+            return (T)ret;
         } catch(IOException ioe) {
             throw new RuntimeException(ioe);
         } catch(ClassNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/serialization/SerializationDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/SerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/SerializationDelegate.java
index b9059b8..e0de7a3 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/SerializationDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/SerializationDelegate.java
@@ -31,5 +31,5 @@ public interface SerializationDelegate {
 
     byte[] serialize(Object object);
 
-    Object deserialize(byte[] bytes);
+    <T> T deserialize(byte[] bytes, Class<T> clazz);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/serialization/Test.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/Test.java b/storm-core/src/jvm/backtype/storm/serialization/Test.java
new file mode 100644
index 0000000..02b2bab
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/serialization/Test.java
@@ -0,0 +1,17 @@
+package backtype.storm.serialization;
+
+import backtype.storm.generated.SupervisorInfo;
+
+/**
+ * Created by pbrahmbhatt on 1/29/15.
+ */
+public class Test {
+
+    public static void main(String[] args) throws Exception {
+//        ThriftSerializationDelegate t = new ThriftSerializationDelegate();
+//        t.serialize(null);
+
+        DefaultSerializationDelegate d = new DefaultSerializationDelegate();
+        System.out.println(d.serialize(null));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegate.java
new file mode 100644
index 0000000..f5d03e4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegate.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.serialization;
+
+import org.apache.thrift.TBase;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+
+import java.util.Map;
+
+public class ThriftSerializationDelegate implements SerializationDelegate {
+
+    @Override
+    public void prepare(Map stormConf) {
+    }
+
+    @Override
+    public byte[] serialize(Object object) {
+        try {
+            return  new TSerializer().serialize((TBase) object);
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public <T> T deserialize(byte[] bytes, Class<T> clazz) {
+        try {
+            TBase instance = (TBase) clazz.newInstance();
+            new TDeserializer().deserialize(instance, bytes);
+            return (T)instance;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java b/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java
new file mode 100644
index 0000000..8b8c95d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.serialization;
+
+import org.apache.thrift.TBase;
+
+import java.util.Map;
+
+public class ThriftSerializationDelegateBridge implements SerializationDelegate {
+    private SerializationDelegate thriftSerializationDelegate = new ThriftSerializationDelegate();
+    private SerializationDelegate defaultSerializationDelegate = new DefaultSerializationDelegate();
+
+    @Override
+    public void prepare(Map stormConf) {
+        this.thriftSerializationDelegate.prepare(stormConf);
+        this.defaultSerializationDelegate.prepare(stormConf);
+    }
+
+    @Override
+    public byte[] serialize(Object object) {
+        if(object instanceof TBase) {
+            return thriftSerializationDelegate.serialize(object);
+        } else {
+            return defaultSerializationDelegate.serialize(object);
+        }
+    }
+
+    @Override
+    public <T> T deserialize(byte[] bytes, Class<T> clazz) {
+        if(TBase.class.isAssignableFrom(clazz)) {
+            return thriftSerializationDelegate.deserialize(bytes, clazz);
+        } else {
+            return defaultSerializationDelegate.deserialize(bytes, clazz);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/utils/LocalState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/LocalState.java b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
index dc64e0f..561988c 100644
--- a/storm-core/src/jvm/backtype/storm/utils/LocalState.java
+++ b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
@@ -64,7 +64,7 @@ public class LocalState {
             if (serialized.length == 0) {
                 LOG.warn("LocalState file '{}' contained no data, resetting state", latestPath);
             } else {
-                result = (Map<Object, Object>) Utils.deserialize(serialized);
+                result = Utils.deserialize(serialized, Map.class);
             }
         }
         return result;

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index 6e8458a..fd56539 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -17,37 +17,18 @@
  */
 package backtype.storm.utils;
 
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.net.URL;
-import java.net.URLDecoder;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.zip.GZIPOutputStream;
-import java.util.zip.GZIPInputStream;
-
+import backtype.storm.Config;
+import backtype.storm.generated.AuthorizationException;
+import backtype.storm.generated.ComponentCommon;
+import backtype.storm.generated.ComponentObject;
+import backtype.storm.generated.StormTopology;
 import backtype.storm.serialization.DefaultSerializationDelegate;
 import backtype.storm.serialization.SerializationDelegate;
+import clojure.lang.IFn;
+import clojure.lang.RT;
+import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.commons.lang.StringUtils;
 import org.apache.thrift.TException;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
@@ -58,14 +39,13 @@ import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.SafeConstructor;
 
-import backtype.storm.Config;
-import backtype.storm.generated.ComponentCommon;
-import backtype.storm.generated.ComponentObject;
-import backtype.storm.generated.StormTopology;
-import backtype.storm.generated.AuthorizationException;
-
-import clojure.lang.IFn;
-import clojure.lang.RT;
+import java.io.*;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.*;
 
 public class Utils {
     private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
@@ -91,8 +71,8 @@ public class Utils {
         return serializationDelegate.serialize(obj);
     }
 
-    public static Object deserialize(byte[] serialized) {
-        return serializationDelegate.deserialize(serialized);
+    public static <T> T deserialize(byte[] serialized, Class<T> clazz) {
+        return serializationDelegate.deserialize(serialized, clazz);
     }
 
     public static <T> String join(Iterable<T> coll, String sep) {
@@ -231,7 +211,7 @@ public class Utils {
 
     public static Object getSetComponentObject(ComponentObject obj) {
         if(obj.getSetField()==ComponentObject._Fields.SERIALIZED_JAVA) {
-            return Utils.deserialize(obj.get_serialized_java());
+            return Utils.deserialize(obj.get_serialized_java(), Serializable.class);
         } else if(obj.getSetField()==ComponentObject._Fields.JAVA_OBJECT) {
             return obj.get_java_object();
         } else {

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/storm/trident/TridentTopology.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/TridentTopology.java b/storm-core/src/jvm/storm/trident/TridentTopology.java
index 8bf2c94..25506c8 100644
--- a/storm-core/src/jvm/storm/trident/TridentTopology.java
+++ b/storm-core/src/jvm/storm/trident/TridentTopology.java
@@ -28,6 +28,8 @@ import backtype.storm.topology.BoltDeclarer;
 import backtype.storm.topology.IRichSpout;
 import backtype.storm.tuple.Fields;
 import backtype.storm.utils.Utils;
+
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -664,7 +666,7 @@ public class TridentTopology {
     private static boolean isIdentityPartition(PartitionNode n) {
         Grouping g = n.thriftGrouping;
         if(g.is_set_custom_serialized()) {
-            CustomStreamGrouping csg = (CustomStreamGrouping) Utils.deserialize(g.get_custom_serialized());
+            CustomStreamGrouping csg = (CustomStreamGrouping) Utils.deserialize(g.get_custom_serialized(), Serializable.class);
             return csg instanceof IdentityGrouping;
         }
         return false;


[04/16] storm git commit: STORM-634: Converting SupervisorInfo, Assignment, StormBase, TopologyStatus, ZKWorkerHeartbeat, ErrorInfo, Credentials to thrift and defaulting the serialization delegate to thrift serialization. Added class as a param to serializatio

Posted by bo...@apache.org.
STORM-634: Converting SupervisorInfo,Assignment,StormBase,TopologyStatus,ZKWorkerHeartbeat,ErrorInfo,Credentials to thrift and defaulting the serialization delegate to thrift serialization. Added class as a param to serialization delegate interface.


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

Branch: refs/heads/master
Commit: 639006432c658226bd33dc2ae607f121f3dc02bb
Parents: a115c9d
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Jan 29 13:46:01 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Tue Feb 3 11:52:15 2015 -0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |    2 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj |    2 +-
 storm-core/src/clj/backtype/storm/cluster.clj   |   47 +-
 storm-core/src/clj/backtype/storm/config.clj    |    7 +-
 storm-core/src/clj/backtype/storm/converter.clj |  200 ++++
 .../src/clj/backtype/storm/daemon/common.clj    |   10 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |    6 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  105 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |    2 +-
 storm-core/src/clj/backtype/storm/stats.clj     |   78 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |    9 +-
 .../storm/coordination/BatchBoltExecutor.java   |    2 +-
 .../backtype/storm/generated/Assignment.java    |  817 +++++++++++++
 .../backtype/storm/generated/ExecutorStats.java |  125 +-
 .../jvm/backtype/storm/generated/NodeInfo.java  |  479 ++++++++
 .../jvm/backtype/storm/generated/StormBase.java | 1078 ++++++++++++++++++
 .../storm/generated/SupervisorInfo.java         | 1030 +++++++++++++++++
 .../storm/generated/TopologyActionOptions.java  |  335 ++++++
 .../storm/generated/TopologyStatus.java         |   67 ++
 .../storm/generated/ZKWorkerHeartbeat.java      |  586 ++++++++++
 .../DefaultSerializationDelegate.java           |   10 +-
 .../GzipBridgeSerializationDelegate.java        |    6 +-
 .../GzipSerializationDelegate.java              |   10 +-
 .../serialization/SerializationDelegate.java    |    2 +-
 .../jvm/backtype/storm/serialization/Test.java  |   17 +
 .../ThriftSerializationDelegate.java            |   52 +
 .../ThriftSerializationDelegateBridge.java      |   51 +
 .../jvm/backtype/storm/utils/LocalState.java    |    2 +-
 .../src/jvm/backtype/storm/utils/Utils.java     |   56 +-
 .../src/jvm/storm/trident/TridentTopology.java  |    4 +-
 storm-core/src/py/storm/ttypes.py               |  796 ++++++++++++-
 storm-core/src/storm.thrift                     |   51 +
 .../test/clj/backtype/storm/cluster_test.clj    |   25 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |    6 +-
 .../GzipBridgeSerializationDelegateTest.java    |    6 +-
 .../ThriftBridgeSerializationDelegateTest.java  |   79 ++
 36 files changed, 5957 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 35d20ff..141e1d3 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -47,7 +47,7 @@ storm.auth.simple-white-list.users: []
 storm.auth.simple-acl.users: []
 storm.auth.simple-acl.users.commands: []
 storm.auth.simple-acl.admins: []
-storm.meta.serialization.delegate: "backtype.storm.serialization.DefaultSerializationDelegate"
+storm.meta.serialization.delegate: "backtype.storm.serialization.ThriftSerializationDelegateBridge"
 
 ### nimbus.* configs are for the master
 nimbus.host: "localhost"

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/clj/backtype/storm/bootstrap.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index c1063cf..bc08e41 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -57,7 +57,7 @@
                      KillOptions SubmitOptions RebalanceOptions JavaObject JavaObjectArg
                      TopologyInitialStatus AuthorizationException]))
      (import (quote [backtype.storm.daemon.common StormBase Assignment
-                     SupervisorInfo WorkerHeartbeat]))
+                     SupervisorInfo]))
      (import (quote [backtype.storm.grouping CustomStreamGrouping]))
      (import (quote [java.io File FileOutputStream FileInputStream]))
      (import (quote [java.util Collection List Random Map HashMap Collections ArrayList LinkedList]))

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index 8ead710..15bf8a3 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -15,12 +15,14 @@
 ;; limitations under the License.
 
 (ns backtype.storm.cluster
-  (:import [org.apache.zookeeper.data Stat ACL Id])
+  (:import [org.apache.zookeeper.data Stat ACL Id]
+           [backtype.storm.generated SupervisorInfo Assignment StormBase ZKWorkerHeartbeat ErrorInfo Credentials]
+           [java.io Serializable])
   (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
   (:import [backtype.storm.utils Utils])
   (:import [java.security MessageDigest])
   (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
-  (:use [backtype.storm util log config])
+  (:use [backtype.storm util log config converter])
   (:require [backtype.storm [zookeeper :as zk]])
   (:require [backtype.storm.daemon [common :as common]]))
 
@@ -228,9 +230,9 @@
       (cb id))))
 
 (defn- maybe-deserialize
-  [ser]
+  [ser clazz]
   (when ser
-    (Utils/deserialize ser)))
+    (Utils/deserialize ser clazz)))
 
 (defstruct TaskError :error :time-secs :host :port)
 
@@ -292,7 +294,7 @@
         [this storm-id callback]
         (when callback
           (swap! assignment-info-callback assoc storm-id callback))
-        (maybe-deserialize (get-data cluster-state (assignment-path storm-id) (not-nil? callback))))
+        (clojurify-assignment (maybe-deserialize (get-data cluster-state (assignment-path storm-id) (not-nil? callback)) Assignment)))
 
       (assignment-info-with-version 
         [this storm-id callback]
@@ -300,7 +302,7 @@
           (swap! assignment-info-with-version-callback assoc storm-id callback))
         (let [{data :data version :version} 
               (get-data-with-version cluster-state (assignment-path storm-id) (not-nil? callback))]
-        {:data (maybe-deserialize data)
+        {:data (clojurify-assignment (maybe-deserialize data Assignment))
          :version version}))
 
       (assignment-version 
@@ -325,7 +327,9 @@
         [this storm-id node port]
         (-> cluster-state
             (get-data (workerbeat-path storm-id node port) false)
-            maybe-deserialize))
+          (maybe-deserialize ZKWorkerHeartbeat)
+          clojurify-zk-worker-hb))
+
 
       (executor-beats
         [this storm-id executor->node+port]
@@ -348,11 +352,12 @@
 
       (supervisor-info
         [this supervisor-id]
-        (maybe-deserialize (get-data cluster-state (supervisor-path supervisor-id) false)))
+        (clojurify-supervisor-info (maybe-deserialize (get-data cluster-state (supervisor-path supervisor-id) false) SupervisorInfo)))
 
       (worker-heartbeat!
         [this storm-id node port info]
-        (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize info) acls))
+        (let [thrift-worker-hb (thriftify-zk-worker-hb info)]
+          (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize thrift-worker-hb) acls)))
 
       (remove-worker-heartbeat!
         [this storm-id node port]
@@ -378,11 +383,13 @@
 
       (supervisor-heartbeat!
         [this supervisor-id info]
-        (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info) acls))
+        (let [thrift-supervisor-info (thriftify-supervisor-info info)]
+          (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize thrift-supervisor-info) acls)))
 
       (activate-storm!
         [this storm-id storm-base]
-        (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base) acls))
+        (let [thrift-storm-base (thriftify-storm-base storm-base)]
+          (set-data cluster-state (storm-path storm-id) (Utils/serialize thrift-storm-base) acls)))
 
       (update-storm!
         [this storm-id new-elems]
@@ -392,6 +399,7 @@
           (set-data cluster-state (storm-path storm-id)
                     (-> base
                         (merge new-elems)
+                        thriftify-storm-base
                         Utils/serialize)
                     acls)))
 
@@ -399,7 +407,7 @@
         [this storm-id callback]
         (when callback
           (swap! storm-base-callback assoc storm-id callback))
-        (maybe-deserialize (get-data cluster-state (storm-path storm-id) (not-nil? callback))))
+        (clojurify-storm-base (maybe-deserialize (get-data cluster-state (storm-path storm-id) (not-nil? callback)) StormBase)))
 
       (remove-storm-base!
         [this storm-id]
@@ -407,7 +415,8 @@
 
       (set-assignment!
         [this storm-id info]
-        (set-data cluster-state (assignment-path storm-id) (Utils/serialize info) acls))
+        (let [thrift-assignment (thriftify-assignment info)]
+          (set-data cluster-state (assignment-path storm-id) (Utils/serialize thrift-assignment) acls)))
 
       (remove-storm!
         [this storm-id]
@@ -418,19 +427,20 @@
       (set-credentials!
          [this storm-id creds topo-conf]
          (let [topo-acls (mk-topo-only-acls topo-conf)
-               path (credentials-path storm-id)]
-           (set-data cluster-state path (Utils/serialize creds) topo-acls)))
+               path (credentials-path storm-id)
+               thriftified-creds (thriftify-credentials creds)]
+           (set-data cluster-state path (Utils/serialize thriftified-creds) topo-acls)))
 
       (credentials
         [this storm-id callback]
         (when callback
           (swap! credentials-callback assoc storm-id callback))
-        (maybe-deserialize (get-data cluster-state (credentials-path storm-id) (not-nil? callback))))
+        (clojurify-crdentials (maybe-deserialize (get-data cluster-state (credentials-path storm-id) (not-nil? callback)) Credentials)))
 
       (report-error
          [this storm-id component-id node port error]
          (let [path (error-path storm-id component-id)
-               data {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port}
+               data (thriftify-error {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port})
                _ (mkdirs cluster-state path acls)
                _ (create-sequential cluster-state (str path "/e") (Utils/serialize data) acls)
                to-kill (->> (get-children cluster-state path false)
@@ -446,7 +456,8 @@
                errors (if (exists-node? cluster-state path false)
                         (dofor [c (get-children cluster-state path false)]
                           (let [data (-> (get-data cluster-state (str path "/" c) false)
-                                         maybe-deserialize)]
+                                       (maybe-deserialize ErrorInfo)
+                                       clojurify-error)]
                             (when data
                               (struct TaskError (:error data) (:time-secs data) (:host data) (:port data))
                               )))

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index 98b1da2..d09b31b 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -15,7 +15,8 @@
 ;; limitations under the License.
 
 (ns backtype.storm.config
-  (:import [java.io FileReader File IOException])
+  (:import [java.io FileReader File IOException]
+           [backtype.storm.generated StormTopology])
   (:import [backtype.storm Config ConfigValidation$FieldValidator])
   (:import [backtype.storm.utils Utils LocalState])
   (:import [org.apache.commons.io FileUtils])
@@ -211,14 +212,14 @@
   (let [stormroot (supervisor-stormdist-root conf storm-id)
         conf-path (supervisor-stormconf-path stormroot)
         topology-path (supervisor-stormcode-path stormroot)]
-    (merge conf (Utils/deserialize (FileUtils/readFileToByteArray (File. conf-path))))
+    (merge conf (Utils/deserialize (FileUtils/readFileToByteArray (File. conf-path)) java.util.Map))
     ))
 
 (defn read-supervisor-topology
   [conf storm-id]
   (let [stormroot (supervisor-stormdist-root conf storm-id)
         topology-path (supervisor-stormcode-path stormroot)]
-    (Utils/deserialize (FileUtils/readFileToByteArray (File. topology-path)))
+    (Utils/deserialize (FileUtils/readFileToByteArray (File. topology-path)) StormTopology)
     ))
 
 (defn worker-user-root [conf]

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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
new file mode 100644
index 0000000..6a9f4a6
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -0,0 +1,200 @@
+(ns backtype.storm.converter
+  (:import [backtype.storm.generated SupervisorInfo NodeInfo Assignment
+            StormBase TopologyStatus ZKWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions TopologyActionOptions])
+  (:use [backtype.storm util stats log])
+  (:require [backtype.storm.daemon [common :as common]]))
+
+(defn thriftify-supervisor-info [supervisor-info]
+  (doto (SupervisorInfo.)
+    (.set_time_secs (long (:time-secs supervisor-info)))
+    (.set_hostname (:hostname supervisor-info))
+    (.set_assignment_id (:assignment-id supervisor-info))
+    (.set_used_ports (map long (:used-ports supervisor-info)))
+    (.set_meta (map long (:meta supervisor-info)))
+    (.set_scheduler_meta (:scheduler-meta supervisor-info))
+    (.set_uptime_secs (long (:uptime-secs supervisor-info)))))
+
+(defn clojurify-supervisor-info [^SupervisorInfo supervisor-info]
+  (if supervisor-info
+    (backtype.storm.daemon.common.SupervisorInfo.
+      (.get_time_secs supervisor-info)
+      (.get_hostname supervisor-info)
+      (.get_assignment_id supervisor-info)
+      (if (.get_used_ports supervisor-info) (into [] (.get_used_ports supervisor-info)))
+      (if (.get_meta supervisor-info) (into [] (.get_meta supervisor-info)))
+      (if (.get_scheduler_meta supervisor-info) (into {} (.get_scheduler_meta supervisor-info)))
+      (.get_uptime_secs supervisor-info))))
+
+(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))))))
+
+(defn clojurify-executor->node_port [executor->node_port]
+  (into {}
+    (map-val
+      (fn [nodeInfo]
+        (concat [(.get_node nodeInfo)] (.get_port nodeInfo))) ;nodeInfo should be converted to [node,port1,port2..]
+      (map-key
+        (fn [list-of-executors]
+          (into [] list-of-executors)) ; list of executors must be coverted to clojure vector to ensure it is sortable.
+        executor->node_port))))
+
+(defn clojurify-assignment [^Assignment assignment]
+  (if assignment
+    (backtype.storm.daemon.common.Assignment.
+      (.get_master_code_dir assignment)
+      (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))))))
+
+(defn convert-to-symbol-from-status [status]
+  (condp = status
+    TopologyStatus/ACTIVE {:type :active}
+    TopologyStatus/INACTIVE {:type :inactive}
+    TopologyStatus/REBALANCING {:type :rebalancing}
+    TopologyStatus/KILLED {:type :killed}
+    nil))
+
+(defn- convert-to-status-from-symbol [status]
+  (if status
+    (condp = (:type status)
+      :active TopologyStatus/ACTIVE
+      :inactive TopologyStatus/INACTIVE
+      :rebalancing TopologyStatus/REBALANCING
+      :killed TopologyStatus/KILLED
+      nil)))
+
+(defn clojurify-rebalance-options [^RebalanceOptions rebalance-options]
+  (-> {:action :rebalance}
+    (assoc-non-nil :delay-secs (if (.is_set_wait_secs rebalance-options) (.get_wait_secs rebalance-options)))
+    (assoc-non-nil :num-workers (if (.is_set_num_workers rebalance-options) (.get_num_workers rebalance-options)))
+    (assoc-non-nil :component->executors (if (.is_set_num_executors rebalance-options) (into {} (.get_num_executors rebalance-options))))))
+
+(defn thriftify-rebalance-options [rebalance-options]
+  (if rebalance-options
+    (let [thrift-rebalance-options (RebalanceOptions.)]
+      (if (:delay-secs rebalance-options)
+        (.set_wait_secs thrift-rebalance-options (int (:delay-secs rebalance-options))))
+      (if (:num-workers rebalance-options)
+        (.set_num_workers thrift-rebalance-options (int (:num-workers rebalance-options))))
+      (if (:component->executors rebalance-options)
+        (.set_num_executors thrift-rebalance-options (map-val int (:component->executors rebalance-options))))
+      thrift-rebalance-options)))
+
+(defn clojurify-kill-options [^KillOptions kill-options]
+  (-> {:action :kill}
+    (assoc-non-nil :delay-secs (if (.is_set_wait_secs kill-options) (.get_wait_secs kill-options)))))
+
+(defn thriftify-kill-options [kill-options]
+  (if kill-options
+    (let [thrift-kill-options (KillOptions.)]
+      (if (:delay-secs kill-options)
+        (.set_wait_secs thrift-kill-options (int (:delay-secs kill-options))))
+      thrift-kill-options)))
+
+(defn thriftify-topology-action-options [storm-base]
+  (if (:topology-action-options storm-base)
+    (let [ topology-action-options (:topology-action-options storm-base)
+           action (:action topology-action-options)
+           thrift-topology-action-options (TopologyActionOptions.)]
+      (if (= action :kill)
+        (.set_kill_options thrift-topology-action-options (thriftify-kill-options topology-action-options)))
+      (if (= action :rebalance)
+        (.set_rebalance_options thrift-topology-action-options (thriftify-rebalance-options topology-action-options)))
+      thrift-topology-action-options)))
+
+(defn clojurify-topology-action-options [^TopologyActionOptions topology-action-options]
+  (if (and topology-action-options (.is_set_kill_options topology-action-options))
+      (clojurify-kill-options (.get_kill_options topology-action-options)))
+  (if (and topology-action-options (.is_set_rebalance_options topology-action-options))
+      (clojurify-rebalance-options (.get_rebalance_options topology-action-options))))
+
+(defn thriftify-storm-base [storm-base]
+  (doto (StormBase.)
+    (.set_name (:storm-name storm-base))
+    (.set_launch_time_secs (int (:launch-time-secs storm-base)))
+    (.set_status (convert-to-status-from-symbol (:status storm-base)))
+    (.set_num_workers (int (:num-workers storm-base)))
+    (.set_component_executors (map-val int (:component->executors storm-base)))
+    (.set_owner (:owner storm-base))
+    (.set_topology_action_options (thriftify-topology-action-options storm-base))
+    (.set_prev_status (convert-to-status-from-symbol (:prev-status storm-base)))))
+
+(defn clojurify-storm-base [^StormBase storm-base]
+  (if storm-base
+    (backtype.storm.daemon.common.StormBase.
+      (.get_name storm-base)
+      (.get_launch_time_secs storm-base)
+      (convert-to-symbol-from-status (.get_status storm-base))
+      (.get_num_workers storm-base)
+      (into {} (.get_component_executors storm-base))
+      (.get_owner storm-base)
+      (clojurify-topology-action-options (.get_topology_action_options storm-base))
+      (convert-to-symbol-from-status (.get_prev_status storm-base)))))
+
+(defn thriftify-stats [stats]
+  (if stats
+    (map-val thriftify-executor-stats
+      (map-key #(ExecutorInfo. (int (first %1)) (int (last %1)))
+        stats))
+    {}))
+
+(defn clojurify-stats [stats]
+  (if stats
+    (map-val clojurify-executor-stats
+      (map-key (fn [x] (list (.get_task_start x) (.get_task_end x)))
+        stats))
+    {}))
+
+(defn clojurify-zk-worker-hb [^ZKWorkerHeartbeat worker-hb]
+  (if worker-hb
+    {:storm-id (.get_storm_id worker-hb)
+     :executor-stats (clojurify-stats (into {} (.get_executor_stats worker-hb)))
+     :uptime (time-delta (.get_time_secs worker-hb))
+     :time-secs (.get_time_secs worker-hb)
+     }
+    {}))
+
+(defn thriftify-zk-worker-hb [worker-hb]
+  (doto (ZKWorkerHeartbeat.)
+    (.set_storm_id (:storm-id worker-hb))
+    (.set_executor_stats (thriftify-stats (:executor-stats worker-hb)))
+    (.set_time_secs (:time-secs worker-hb))))
+
+(defn clojurify-error [^ErrorInfo error]
+  (if error
+    {
+      :error (.get_error error)
+      :time-secs (.get_error_time_secs error)
+      :host (.get_host error)
+      :port (.get_port error)
+      }
+    ))
+
+(defn thriftify-error [error]
+  (doto (ErrorInfo. (:error error) (:time-secs error))
+    (.set_host (:host error))
+    (.set_port (:port error))))
+
+(defn thriftify-credentials [credentials]
+    (doto (Credentials.)
+      (.set_creds (if credentials credentials {}))))
+
+(defn clojurify-crdentials [^Credentials credentials]
+  (if credentials
+    (into {} (.get_creds credentials))
+    nil
+    ))
+

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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 f091dfb..c33609d 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 @@
 
 
 ;; component->executors is a map from spout/bolt id to number of executors for that component
-(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner])
+(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner topology-action-options prev-status])
 
 (defrecord SupervisorInfo [time-secs hostname assignment-id used-ports meta scheduler-meta uptime-secs])
 
@@ -65,9 +65,11 @@
 (def LS-LOCAL-ASSIGNMENTS "local-assignments")
 (def LS-APPROVED-WORKERS "approved-workers")
 
-
-
-(defrecord WorkerHeartbeat [time-secs storm-id executors port])
+(defn mk-local-worker-heartbeat [time-secs storm-id executors port]
+  {:time-secs time-secs
+   :storm-id storm-id
+   :executors executors
+   :port port})
 
 (defrecord ExecutorStats [^long processed
                           ^long acked

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index a6e606d..a1ecc4a 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -16,7 +16,9 @@
 (ns backtype.storm.daemon.executor
   (:use [backtype.storm.daemon common])
   (:use [backtype.storm bootstrap])
-  (:import [backtype.storm ICredentialsListener])
+  (:import [backtype.storm ICredentialsListener]
+           [backtype.storm.generated Grouping]
+           [java.io Serializable])
   (:import [backtype.storm.hooks ITaskHook])
   (:import [backtype.storm.tuple Tuple])
   (:import [backtype.storm.spout ISpoutWaitStrategy])
@@ -86,7 +88,7 @@
         (let [grouping (thrift/instantiate-java-object (.get_custom_object thrift-grouping))]
           (mk-custom-grouper grouping context component-id stream-id target-tasks))
       :custom-serialized
-        (let [grouping (Utils/deserialize (.get_custom_serialized thrift-grouping))]
+        (let [grouping (Utils/deserialize (.get_custom_serialized thrift-grouping) Serializable)]
           (mk-custom-grouper grouping context component-id stream-id target-tasks))
       :direct
         :direct

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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 b2cb96a..77f59ad 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -15,7 +15,8 @@
 ;; limitations under the License.
 (ns backtype.storm.daemon.nimbus
   (:import [java.nio ByteBuffer]
-           [java.util Collections])
+           [java.util Collections]
+           [backtype.storm.generated StormTopology])
   (:import [java.io FileNotFoundException])
   (:import [java.nio.channels Channels WritableByteChannel])
   (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils])
@@ -99,15 +100,7 @@
            (Utils/deserialize
             (FileUtils/readFileToByteArray
              (File. (master-stormconf-path stormroot))
-             )))))
-
-(defn set-topology-status! [nimbus storm-id status]
-  (let [storm-cluster-state (:storm-cluster-state nimbus)]
-   (.update-storm! storm-cluster-state
-                   storm-id
-                   {:status status})
-   (log-message "Updated " storm-id " with status " status)
-   ))
+             ) java.util.Map))))
 
 (declare delay-event)
 (declare mk-assignments)
@@ -122,8 +115,9 @@
                    storm-id
                    delay
                    :remove)
-      {:type :killed
-       :kill-time-secs delay})
+      {
+        :status {:type :killed}
+        :topology-action-options {:delay-secs delay :action :kill}})
     ))
 
 (defn rebalance-transition [nimbus storm-id status]
@@ -136,24 +130,24 @@
                    storm-id
                    delay
                    :do-rebalance)
-      {:type :rebalancing
-       :delay-secs delay
-       :old-status status
-       :num-workers num-workers
-       :executor-overrides executor-overrides
+      {:status {:type :rebalancing}
+       :prev-status status
+       :topology-action-options (-> {:delay-secs delay :action :rebalance}
+                                  (assoc-non-nil :num-workers num-workers)
+                                  (assoc-non-nil :component->executors executor-overrides))
        })))
 
-(defn do-rebalance [nimbus storm-id status]
-  (.update-storm! (:storm-cluster-state nimbus)
-                  storm-id
-                  (assoc-non-nil
-                    {:component->executors (:executor-overrides status)}
-                    :num-workers
-                    (:num-workers status)))
+(defn do-rebalance [nimbus storm-id status storm-base]
+  (let [rebalance-options (:topology-action-options storm-base)]
+    (.update-storm! (:storm-cluster-state nimbus)
+      storm-id
+        (-> {:topology-action-options nil}
+          (assoc-non-nil :component->executors (:component->executors rebalance-options))
+          (assoc-non-nil :num-workers (:num-workers rebalance-options)))))
   (mk-assignments nimbus :scratch-topology-id storm-id))
 
-(defn state-transitions [nimbus storm-id status]
-  {:active {:inactivate :inactive            
+(defn state-transitions [nimbus storm-id status storm-base]
+  {:active {:inactivate :inactive
             :activate nil
             :rebalance (rebalance-transition nimbus storm-id status)
             :kill (kill-transition nimbus storm-id)
@@ -165,7 +159,7 @@
               }
    :killed {:startup (fn [] (delay-event nimbus
                                          storm-id
-                                         (:kill-time-secs status)
+                                         (:delay-secs storm-base)
                                          :remove)
                              nil)
             :kill (kill-transition nimbus storm-id)
@@ -177,18 +171,15 @@
             }
    :rebalancing {:startup (fn [] (delay-event nimbus
                                               storm-id
-                                              (:delay-secs status)
+                                              (:delay-secs storm-base)
                                               :do-rebalance)
                                  nil)
                  :kill (kill-transition nimbus storm-id)
                  :do-rebalance (fn []
-                                 (do-rebalance nimbus storm-id status)
-                                 (:old-status status))
+                                 (do-rebalance nimbus storm-id status storm-base)
+                                 (:type (:prev-status storm-base)))
                  }})
 
-(defn topology-status [nimbus storm-id]
-  (-> nimbus :storm-cluster-state (.storm-base storm-id nil) :status))
-
 (defn transition!
   ([nimbus storm-id event]
      (transition! nimbus storm-id event false))
@@ -196,7 +187,8 @@
      (locking (:submit-lock nimbus)
        (let [system-events #{:startup}
              [event & event-args] (if (keyword? event) [event] event)
-             status (topology-status nimbus storm-id)]
+             storm-base (-> nimbus :storm-cluster-state  (.storm-base storm-id nil))
+             status (:status storm-base)]
          ;; handles the case where event was scheduled but topology has been removed
          (if-not status
            (log-message "Cannot apply event " event " to " storm-id " because topology no longer exists")
@@ -212,19 +204,20 @@
                                          (log-message msg))
                                        nil))
                                  )))
-                 transition (-> (state-transitions nimbus storm-id status)
+                 transition (-> (state-transitions nimbus storm-id status storm-base)
                                 (get (:type status))
                                 (get-event event))
                  transition (if (or (nil? transition)
                                     (keyword? transition))
                               (fn [] transition)
                               transition)
-                 new-status (apply transition event-args)
-                 new-status (if (keyword? new-status)
-                              {:type new-status}
-                              new-status)]
-             (when new-status
-               (set-topology-status! nimbus storm-id new-status)))))
+                 storm-base-updates (apply transition event-args)
+                 storm-base-updates (if (keyword? storm-base-updates) ;if it's just a symbol, that just indicates new status.
+                                      {:status {:type storm-base-updates}}
+                                      storm-base-updates)]
+
+             (when storm-base-updates
+               (.update-storm! (:storm-cluster-state nimbus) storm-id storm-base-updates)))))
        )))
 
 (defn transition-name! [nimbus storm-name event & args]
@@ -296,7 +289,7 @@
   [nimbus topologies missing-assignment-topologies]
   (let [storm-cluster-state (:storm-cluster-state nimbus)
         ^INimbus inimbus (:inimbus nimbus)
-        
+
         supervisor-infos (all-supervisor-info storm-cluster-state nil)
 
         supervisor-details (dofor [[id info] supervisor-infos]
@@ -327,7 +320,7 @@
     (Utils/deserialize
       (FileUtils/readFileToByteArray
         (File. (master-stormcode-path stormroot))
-        ))))
+        ) StormTopology)))
 
 (declare compute-executor->component)
 
@@ -468,7 +461,7 @@
                                          all-executors
                                          (set (alive-executors nimbus topology-details all-executors assignment)))]]
              {tid alive-executors})))
-  
+
 (defn- compute-supervisor->dead-ports [nimbus existing-assignments topology->executors topology->alive-executors]
   (let [dead-slots (into [] (for [[tid assignment] existing-assignments
                                   :let [all-executors (topology->executors tid)
@@ -514,7 +507,7 @@
                                                                   ((fn [ports] (map int ports))))
                                                     supervisor-details (SupervisorDetails. sid hostname scheduler-meta all-ports)]]
                                           {sid supervisor-details}))]
-    (merge all-supervisor-details 
+    (merge all-supervisor-details
            (into {}
               (for [[sid ports] nonexistent-supervisor-slots]
                 [sid (SupervisorDetails. sid nil ports)]))
@@ -576,7 +569,7 @@
         topology->scheduler-assignment (compute-topology->scheduler-assignment nimbus
                                                                                existing-assignments
                                                                                topology->alive-executors)
-                                                                               
+
         missing-assignment-topologies (->> topologies
                                            .getTopologies
                                            (map (memfn getId))
@@ -594,7 +587,7 @@
         all-scheduling-slots (->> (all-scheduling-slots nimbus topologies missing-assignment-topologies)
                                   (map (fn [[node-id port]] {node-id #{port}}))
                                   (apply merge-with set/union))
-        
+
         supervisors (read-all-supervisor-details nimbus all-scheduling-slots supervisor->dead-ports)
         cluster (Cluster. (:inimbus nimbus) supervisors topology->scheduler-assignment)
 
@@ -659,7 +652,7 @@
 (defnk mk-assignments [nimbus :scratch-topology-id nil]
   (let [conf (:conf nimbus)
         storm-cluster-state (:storm-cluster-state nimbus)
-        ^INimbus inimbus (:inimbus nimbus) 
+        ^INimbus inimbus (:inimbus nimbus)
         ;; read all the topologies
         topology-ids (.active-storms storm-cluster-state)
         topologies (into {} (for [tid topology-ids]
@@ -679,13 +672,13 @@
                                        existing-assignments
                                        topologies
                                        scratch-topology-id)
-        
+
         topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
-        
+
         now-secs (current-time-secs)
-        
+
         basic-supervisor-details-map (basic-supervisor-details-map storm-cluster-state)
-        
+
         ;; construct the final Assignments by adding start-times etc into it
         new-assignments (into {} (for [[topology-id executor->node+port] topology->executor->node+port
                                         :let [existing-assignment (get existing-assignments topology-id)
@@ -723,14 +716,14 @@
     (->> new-assignments
           (map (fn [[topology-id assignment]]
             (let [existing-assignment (get existing-assignments topology-id)]
-              [topology-id (map to-worker-slot (newly-added-slots existing-assignment assignment))] 
+              [topology-id (map to-worker-slot (newly-added-slots existing-assignment assignment))]
               )))
           (into {})
           (.assignSlots inimbus topologies))
     ))
 
 (defn- start-storm [nimbus storm-name storm-id topology-initial-status]
-  {:pre [(#{:active :inactive} topology-initial-status)]}                
+  {:pre [(#{:active :inactive} topology-initial-status)]}
   (let [storm-cluster-state (:storm-cluster-state nimbus)
         conf (:conf nimbus)
         storm-conf (read-storm-conf conf storm-id)
@@ -744,7 +737,9 @@
                                   {:type topology-initial-status}
                                   (storm-conf TOPOLOGY-WORKERS)
                                   num-executors
-                                  (storm-conf TOPOLOGY-SUBMITTER-USER)))))
+                                  (storm-conf TOPOLOGY-SUBMITTER-USER)
+                                  nil
+                                  nil))))
 
 ;; Master:
 ;; job submit:

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index e717ce4..b13e8a7 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -61,7 +61,7 @@
 
 (defn do-heartbeat [worker]
   (let [conf (:conf worker)
-        hb (WorkerHeartbeat.
+        hb (mk-local-worker-heartbeat
              (current-time-secs)
              (:storm-id worker)
              (:executors worker)

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/clj/backtype/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/stats.clj b/storm-core/src/clj/backtype/storm/stats.clj
index b872c6f..db8930c 100644
--- a/storm-core/src/clj/backtype/storm/stats.clj
+++ b/storm-core/src/clj/backtype/storm/stats.clj
@@ -19,7 +19,7 @@
             NotAliveException AlreadyAliveException InvalidTopologyException GlobalStreamId
             ClusterSummary TopologyInfo TopologySummary ExecutorSummary ExecutorStats ExecutorSpecificStats
             SpoutStats BoltStats ErrorInfo SupervisorSummary])
-  (:use [backtype.storm util])
+  (:use [backtype.storm util log])
   (:use [clojure.math.numeric-tower :only [ceil]]))
 
 ;;TODO: consider replacing this with some sort of RRD
@@ -301,42 +301,76 @@
   (value-bolt-stats! stats))
 
 (defmulti thriftify-specific-stats :type)
+(defmulti clojurify-specific-stats class-selector)
 
 (defn window-set-converter
-  ([stats key-fn]
-   ;; make the first key a string,
-   (into {}
-         (for [[k v] stats]
-           [(str k)
-            (into {} (for [[k2 v2] v]
-                       [(key-fn k2) v2]))])))
-  ([stats]
-   (window-set-converter stats identity)))
+  ([stats key-fn first-key-fun]
+    (into {}
+      (for [[k v] stats]
+        ;apply the first-key-fun only to first key.
+        [(first-key-fun k)
+         (into {} (for [[k2 v2] v]
+                    [(key-fn k2) v2]))])))
+  ([stats first-key-fun]
+    (window-set-converter stats identity first-key-fun)))
 
 (defn to-global-stream-id
   [[component stream]]
   (GlobalStreamId. component stream))
 
+(defn from-global-stream-id [global-stream-id]
+  [(.get_componentId global-stream-id) (.get_streamId global-stream-id)])
+
+(defmethod clojurify-specific-stats BoltStats [^BoltStats stats]
+  [(window-set-converter (.get_acked stats) from-global-stream-id symbol)
+   (window-set-converter (.get_failed stats) from-global-stream-id symbol)
+   (window-set-converter (.get_process_ms_avg stats) from-global-stream-id symbol)
+   (window-set-converter (.get_executed stats) from-global-stream-id symbol)
+   (window-set-converter (.get_execute_ms_avg stats) from-global-stream-id symbol)])
+
+(defmethod clojurify-specific-stats SpoutStats [^SpoutStats stats]
+  [(window-set-converter (.get_acked stats) symbol)
+   (window-set-converter (.get_failed stats) symbol)
+   (window-set-converter (.get_complete_ms_avg stats) symbol)])
+
+
+(defn clojurify-executor-stats
+  [^ExecutorStats stats]
+  (let [ specific-stats (.get_specific stats)
+         is_bolt? (.is_set_bolt specific-stats)
+         specific-stats (if is_bolt? (.get_bolt specific-stats) (.get_spout specific-stats))
+         specific-stats (clojurify-specific-stats specific-stats)
+         common-stats (CommonStats. (window-set-converter (.get_emitted stats) symbol) (window-set-converter (.get_transferred stats) symbol) (.get_rate stats))]
+    (if is_bolt?
+      ; worker heart beat does not store the BoltExecutorStats or SpoutExecutorStats , instead it stores the result returned by render-stats!
+      ; which flattens the BoltExecutorStats/SpoutExecutorStats by extracting values from all atoms and merging all values inside :common to top
+      ;level map we are pretty much doing the same here.
+      (dissoc (merge common-stats {:type :bolt}  (apply ->BoltExecutorStats (into [nil] specific-stats))) :common)
+      (dissoc (merge common-stats {:type :spout} (apply ->SpoutExecutorStats (into [nil] specific-stats))) :common)
+      )))
+
 (defmethod thriftify-specific-stats :bolt
   [stats]
   (ExecutorSpecificStats/bolt
     (BoltStats.
-      (window-set-converter (:acked stats) to-global-stream-id)
-      (window-set-converter (:failed stats) to-global-stream-id)
-      (window-set-converter (:process-latencies stats) to-global-stream-id)
-      (window-set-converter (:executed stats) to-global-stream-id)
-      (window-set-converter (:execute-latencies stats) to-global-stream-id))))
+      (window-set-converter (:acked stats) to-global-stream-id str)
+      (window-set-converter (:failed stats) to-global-stream-id str)
+      (window-set-converter (:process-latencies stats) to-global-stream-id str)
+      (window-set-converter (:executed stats) to-global-stream-id str)
+      (window-set-converter (:execute-latencies stats) to-global-stream-id str))))
 
 (defmethod thriftify-specific-stats :spout
   [stats]
   (ExecutorSpecificStats/spout
-    (SpoutStats. (window-set-converter (:acked stats))
-                 (window-set-converter (:failed stats))
-                 (window-set-converter (:complete-latencies stats)))))
+    (SpoutStats. (window-set-converter (:acked stats) str)
+      (window-set-converter (:failed stats) str)
+      (window-set-converter (:complete-latencies stats) str))))
 
 (defn thriftify-executor-stats
   [stats]
-  (let [specific-stats (thriftify-specific-stats stats)]
-    (ExecutorStats. (window-set-converter (:emitted stats))
-                    (window-set-converter (:transferred stats))
-                    specific-stats)))
+  (let [specific-stats (thriftify-specific-stats stats)
+        rate (:rate stats)]
+    (ExecutorStats. (window-set-converter (:emitted stats) str)
+      (window-set-converter (:transferred stats) str)
+      specific-stats
+      rate)))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/clj/backtype/storm/thrift.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/thrift.clj b/storm-core/src/clj/backtype/storm/thrift.clj
index ce0a5ff..5bc1150 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -15,12 +15,14 @@
 ;; limitations under the License.
 
 (ns backtype.storm.thrift
-  (:import [java.util HashMap])
+  (:import [java.util HashMap]
+           [java.io Serializable]
+           [backtype.storm.generated NodeInfo Assignment])
   (:import [backtype.storm.generated JavaObject Grouping Nimbus StormTopology
             StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface
             ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
             GlobalStreamId ComponentObject ComponentObject$_Fields
-            ShellComponent])
+            ShellComponent SupervisorInfo])
   (:import [backtype.storm.utils Utils NimbusClient])
   (:import [backtype.storm Constants])
   (:import [backtype.storm.grouping CustomStreamGrouping])
@@ -155,7 +157,7 @@
   [^ComponentObject obj]
   (when (not= (.getSetField obj) ComponentObject$_Fields/SERIALIZED_JAVA)
     (throw (RuntimeException. "Cannot deserialize non-java-serialized object")))
-  (Utils/deserialize (.get_serialized_java obj)))
+  (Utils/deserialize (.get_serialized_java obj) Serializable))
 
 (defn serialize-component-object
   [obj]
@@ -271,3 +273,4 @@
 (def SPOUT-FIELDS
   [StormTopology$_Fields/SPOUTS
    StormTopology$_Fields/STATE_SPOUTS])
+

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
index b9b97e9..3d3208b 100644
--- a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
@@ -103,6 +103,6 @@ public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCa
     }
     
     private IBatchBolt newTransactionalBolt() {
-        return (IBatchBolt) Utils.deserialize(_boltSer);
+        return Utils.deserialize(_boltSer, IBatchBolt.class);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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
new file mode 100644
index 0000000..b6ade94
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
@@ -0,0 +1,817 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment");
+
+  private static final org.apache.thrift.protocol.TField MASTER_CODE_DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("master_code_dir", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField NODE_HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("node_host", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_NODE_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_node_port", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_START_TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_start_time_secs", org.apache.thrift.protocol.TType.MAP, (short)4);
+
+  private String master_code_dir; // required
+  private Map<String,String> node_host; // required
+  private Map<List<Long>,NodeInfo> executor_node_port; // required
+  private Map<List<Long>,Long> executor_start_time_secs; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    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");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MASTER_CODE_DIR
+          return MASTER_CODE_DIR;
+        case 2: // NODE_HOST
+          return NODE_HOST;
+        case 3: // EXECUTOR_NODE_PORT
+          return EXECUTOR_NODE_PORT;
+        case 4: // EXECUTOR_START_TIME_SECS
+          return EXECUTOR_START_TIME_SECS;
+        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.MASTER_CODE_DIR, new org.apache.thrift.meta_data.FieldMetaData("master_code_dir", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NODE_HOST, new org.apache.thrift.meta_data.FieldMetaData("node_host", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.EXECUTOR_NODE_PORT, new org.apache.thrift.meta_data.FieldMetaData("executor_node_port", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)),
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NodeInfo.class))));
+    tmpMap.put(_Fields.EXECUTOR_START_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("executor_start_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)),
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Assignment.class, metaDataMap);
+  }
+
+  public Assignment() {
+    this.node_host = new HashMap<String,String>();
+
+    this.executor_node_port = new HashMap<List<Long>,NodeInfo>();
+
+    this.executor_start_time_secs = new HashMap<List<Long>,Long>();
+
+  }
+
+  public Assignment(
+    String master_code_dir)
+  {
+    this();
+    this.master_code_dir = master_code_dir;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Assignment(Assignment other) {
+    if (other.is_set_master_code_dir()) {
+      this.master_code_dir = other.master_code_dir;
+    }
+    if (other.is_set_node_host()) {
+      Map<String,String> __this__node_host = new HashMap<String,String>();
+      for (Map.Entry<String, String> other_element : other.node_host.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        String other_element_value = other_element.getValue();
+
+        String __this__node_host_copy_key = other_element_key;
+
+        String __this__node_host_copy_value = other_element_value;
+
+        __this__node_host.put(__this__node_host_copy_key, __this__node_host_copy_value);
+      }
+      this.node_host = __this__node_host;
+    }
+    if (other.is_set_executor_node_port()) {
+      Map<List<Long>,NodeInfo> __this__executor_node_port = new HashMap<List<Long>,NodeInfo>();
+      for (Map.Entry<List<Long>, NodeInfo> other_element : other.executor_node_port.entrySet()) {
+
+        List<Long> other_element_key = other_element.getKey();
+        NodeInfo other_element_value = other_element.getValue();
+
+        List<Long> __this__executor_node_port_copy_key = new ArrayList<Long>();
+        for (Long other_element_key_element : other_element_key) {
+          __this__executor_node_port_copy_key.add(other_element_key_element);
+        }
+
+        NodeInfo __this__executor_node_port_copy_value = new NodeInfo(other_element_value);
+
+        __this__executor_node_port.put(__this__executor_node_port_copy_key, __this__executor_node_port_copy_value);
+      }
+      this.executor_node_port = __this__executor_node_port;
+    }
+    if (other.is_set_executor_start_time_secs()) {
+      Map<List<Long>,Long> __this__executor_start_time_secs = new HashMap<List<Long>,Long>();
+      for (Map.Entry<List<Long>, Long> other_element : other.executor_start_time_secs.entrySet()) {
+
+        List<Long> other_element_key = other_element.getKey();
+        Long other_element_value = other_element.getValue();
+
+        List<Long> __this__executor_start_time_secs_copy_key = new ArrayList<Long>();
+        for (Long other_element_key_element : other_element_key) {
+          __this__executor_start_time_secs_copy_key.add(other_element_key_element);
+        }
+
+        Long __this__executor_start_time_secs_copy_value = other_element_value;
+
+        __this__executor_start_time_secs.put(__this__executor_start_time_secs_copy_key, __this__executor_start_time_secs_copy_value);
+      }
+      this.executor_start_time_secs = __this__executor_start_time_secs;
+    }
+  }
+
+  public Assignment deepCopy() {
+    return new Assignment(this);
+  }
+
+  @Override
+  public void clear() {
+    this.master_code_dir = null;
+    this.node_host = new HashMap<String,String>();
+
+    this.executor_node_port = new HashMap<List<Long>,NodeInfo>();
+
+    this.executor_start_time_secs = new HashMap<List<Long>,Long>();
+
+  }
+
+  public String get_master_code_dir() {
+    return this.master_code_dir;
+  }
+
+  public void set_master_code_dir(String master_code_dir) {
+    this.master_code_dir = master_code_dir;
+  }
+
+  public void unset_master_code_dir() {
+    this.master_code_dir = null;
+  }
+
+  /** Returns true if field master_code_dir is set (has been assigned a value) and false otherwise */
+  public boolean is_set_master_code_dir() {
+    return this.master_code_dir != null;
+  }
+
+  public void set_master_code_dir_isSet(boolean value) {
+    if (!value) {
+      this.master_code_dir = null;
+    }
+  }
+
+  public int get_node_host_size() {
+    return (this.node_host == null) ? 0 : this.node_host.size();
+  }
+
+  public void put_to_node_host(String key, String val) {
+    if (this.node_host == null) {
+      this.node_host = new HashMap<String,String>();
+    }
+    this.node_host.put(key, val);
+  }
+
+  public Map<String,String> get_node_host() {
+    return this.node_host;
+  }
+
+  public void set_node_host(Map<String,String> node_host) {
+    this.node_host = node_host;
+  }
+
+  public void unset_node_host() {
+    this.node_host = null;
+  }
+
+  /** Returns true if field node_host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_node_host() {
+    return this.node_host != null;
+  }
+
+  public void set_node_host_isSet(boolean value) {
+    if (!value) {
+      this.node_host = null;
+    }
+  }
+
+  public int get_executor_node_port_size() {
+    return (this.executor_node_port == null) ? 0 : this.executor_node_port.size();
+  }
+
+  public void put_to_executor_node_port(List<Long> key, NodeInfo val) {
+    if (this.executor_node_port == null) {
+      this.executor_node_port = new HashMap<List<Long>,NodeInfo>();
+    }
+    this.executor_node_port.put(key, val);
+  }
+
+  public Map<List<Long>,NodeInfo> get_executor_node_port() {
+    return this.executor_node_port;
+  }
+
+  public void set_executor_node_port(Map<List<Long>,NodeInfo> executor_node_port) {
+    this.executor_node_port = executor_node_port;
+  }
+
+  public void unset_executor_node_port() {
+    this.executor_node_port = null;
+  }
+
+  /** Returns true if field executor_node_port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_node_port() {
+    return this.executor_node_port != null;
+  }
+
+  public void set_executor_node_port_isSet(boolean value) {
+    if (!value) {
+      this.executor_node_port = null;
+    }
+  }
+
+  public int get_executor_start_time_secs_size() {
+    return (this.executor_start_time_secs == null) ? 0 : this.executor_start_time_secs.size();
+  }
+
+  public void put_to_executor_start_time_secs(List<Long> key, long val) {
+    if (this.executor_start_time_secs == null) {
+      this.executor_start_time_secs = new HashMap<List<Long>,Long>();
+    }
+    this.executor_start_time_secs.put(key, val);
+  }
+
+  public Map<List<Long>,Long> get_executor_start_time_secs() {
+    return this.executor_start_time_secs;
+  }
+
+  public void set_executor_start_time_secs(Map<List<Long>,Long> executor_start_time_secs) {
+    this.executor_start_time_secs = executor_start_time_secs;
+  }
+
+  public void unset_executor_start_time_secs() {
+    this.executor_start_time_secs = null;
+  }
+
+  /** Returns true if field executor_start_time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_start_time_secs() {
+    return this.executor_start_time_secs != null;
+  }
+
+  public void set_executor_start_time_secs_isSet(boolean value) {
+    if (!value) {
+      this.executor_start_time_secs = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MASTER_CODE_DIR:
+      if (value == null) {
+        unset_master_code_dir();
+      } else {
+        set_master_code_dir((String)value);
+      }
+      break;
+
+    case NODE_HOST:
+      if (value == null) {
+        unset_node_host();
+      } else {
+        set_node_host((Map<String,String>)value);
+      }
+      break;
+
+    case EXECUTOR_NODE_PORT:
+      if (value == null) {
+        unset_executor_node_port();
+      } else {
+        set_executor_node_port((Map<List<Long>,NodeInfo>)value);
+      }
+      break;
+
+    case EXECUTOR_START_TIME_SECS:
+      if (value == null) {
+        unset_executor_start_time_secs();
+      } else {
+        set_executor_start_time_secs((Map<List<Long>,Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MASTER_CODE_DIR:
+      return get_master_code_dir();
+
+    case NODE_HOST:
+      return get_node_host();
+
+    case EXECUTOR_NODE_PORT:
+      return get_executor_node_port();
+
+    case EXECUTOR_START_TIME_SECS:
+      return get_executor_start_time_secs();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MASTER_CODE_DIR:
+      return is_set_master_code_dir();
+    case NODE_HOST:
+      return is_set_node_host();
+    case EXECUTOR_NODE_PORT:
+      return is_set_executor_node_port();
+    case EXECUTOR_START_TIME_SECS:
+      return is_set_executor_start_time_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Assignment)
+      return this.equals((Assignment)that);
+    return false;
+  }
+
+  public boolean equals(Assignment that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_master_code_dir = true && this.is_set_master_code_dir();
+    boolean that_present_master_code_dir = true && that.is_set_master_code_dir();
+    if (this_present_master_code_dir || that_present_master_code_dir) {
+      if (!(this_present_master_code_dir && that_present_master_code_dir))
+        return false;
+      if (!this.master_code_dir.equals(that.master_code_dir))
+        return false;
+    }
+
+    boolean this_present_node_host = true && this.is_set_node_host();
+    boolean that_present_node_host = true && that.is_set_node_host();
+    if (this_present_node_host || that_present_node_host) {
+      if (!(this_present_node_host && that_present_node_host))
+        return false;
+      if (!this.node_host.equals(that.node_host))
+        return false;
+    }
+
+    boolean this_present_executor_node_port = true && this.is_set_executor_node_port();
+    boolean that_present_executor_node_port = true && that.is_set_executor_node_port();
+    if (this_present_executor_node_port || that_present_executor_node_port) {
+      if (!(this_present_executor_node_port && that_present_executor_node_port))
+        return false;
+      if (!this.executor_node_port.equals(that.executor_node_port))
+        return false;
+    }
+
+    boolean this_present_executor_start_time_secs = true && this.is_set_executor_start_time_secs();
+    boolean that_present_executor_start_time_secs = true && that.is_set_executor_start_time_secs();
+    if (this_present_executor_start_time_secs || that_present_executor_start_time_secs) {
+      if (!(this_present_executor_start_time_secs && that_present_executor_start_time_secs))
+        return false;
+      if (!this.executor_start_time_secs.equals(that.executor_start_time_secs))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_master_code_dir = true && (is_set_master_code_dir());
+    builder.append(present_master_code_dir);
+    if (present_master_code_dir)
+      builder.append(master_code_dir);
+
+    boolean present_node_host = true && (is_set_node_host());
+    builder.append(present_node_host);
+    if (present_node_host)
+      builder.append(node_host);
+
+    boolean present_executor_node_port = true && (is_set_executor_node_port());
+    builder.append(present_executor_node_port);
+    if (present_executor_node_port)
+      builder.append(executor_node_port);
+
+    boolean present_executor_start_time_secs = true && (is_set_executor_start_time_secs());
+    builder.append(present_executor_start_time_secs);
+    if (present_executor_start_time_secs)
+      builder.append(executor_start_time_secs);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(Assignment other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    Assignment typedOther = (Assignment)other;
+
+    lastComparison = Boolean.valueOf(is_set_master_code_dir()).compareTo(typedOther.is_set_master_code_dir());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_master_code_dir()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.master_code_dir, typedOther.master_code_dir);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_node_host()).compareTo(typedOther.is_set_node_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_node_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node_host, typedOther.node_host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executor_node_port()).compareTo(typedOther.is_set_executor_node_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_node_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_node_port, typedOther.executor_node_port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executor_start_time_secs()).compareTo(typedOther.is_set_executor_start_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_start_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_start_time_secs, typedOther.executor_start_time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        break;
+      }
+      switch (field.id) {
+        case 1: // MASTER_CODE_DIR
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.master_code_dir = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // NODE_HOST
+          if (field.type == org.apache.thrift.protocol.TType.MAP) {
+            {
+              org.apache.thrift.protocol.TMap _map185 = iprot.readMapBegin();
+              this.node_host = new HashMap<String,String>(2*_map185.size);
+              for (int _i186 = 0; _i186 < _map185.size; ++_i186)
+              {
+                String _key187; // required
+                String _val188; // required
+                _key187 = iprot.readString();
+                _val188 = iprot.readString();
+                this.node_host.put(_key187, _val188);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // EXECUTOR_NODE_PORT
+          if (field.type == org.apache.thrift.protocol.TType.MAP) {
+            {
+              org.apache.thrift.protocol.TMap _map189 = iprot.readMapBegin();
+              this.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map189.size);
+              for (int _i190 = 0; _i190 < _map189.size; ++_i190)
+              {
+                List<Long> _key191; // required
+                NodeInfo _val192; // required
+                {
+                  org.apache.thrift.protocol.TList _list193 = iprot.readListBegin();
+                  _key191 = new ArrayList<Long>(_list193.size);
+                  for (int _i194 = 0; _i194 < _list193.size; ++_i194)
+                  {
+                    long _elem195; // required
+                    _elem195 = iprot.readI64();
+                    _key191.add(_elem195);
+                  }
+                  iprot.readListEnd();
+                }
+                _val192 = new NodeInfo();
+                _val192.read(iprot);
+                this.executor_node_port.put(_key191, _val192);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // EXECUTOR_START_TIME_SECS
+          if (field.type == org.apache.thrift.protocol.TType.MAP) {
+            {
+              org.apache.thrift.protocol.TMap _map196 = iprot.readMapBegin();
+              this.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map196.size);
+              for (int _i197 = 0; _i197 < _map196.size; ++_i197)
+              {
+                List<Long> _key198; // required
+                long _val199; // required
+                {
+                  org.apache.thrift.protocol.TList _list200 = iprot.readListBegin();
+                  _key198 = new ArrayList<Long>(_list200.size);
+                  for (int _i201 = 0; _i201 < _list200.size; ++_i201)
+                  {
+                    long _elem202; // required
+                    _elem202 = iprot.readI64();
+                    _key198.add(_elem202);
+                  }
+                  iprot.readListEnd();
+                }
+                _val199 = iprot.readI64();
+                this.executor_start_time_secs.put(_key198, _val199);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.master_code_dir != null) {
+      oprot.writeFieldBegin(MASTER_CODE_DIR_FIELD_DESC);
+      oprot.writeString(this.master_code_dir);
+      oprot.writeFieldEnd();
+    }
+    if (this.node_host != null) {
+      if (is_set_node_host()) {
+        oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, this.node_host.size()));
+          for (Map.Entry<String, String> _iter203 : this.node_host.entrySet())
+          {
+            oprot.writeString(_iter203.getKey());
+            oprot.writeString(_iter203.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.executor_node_port != null) {
+      if (is_set_executor_node_port()) {
+        oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, this.executor_node_port.size()));
+          for (Map.Entry<List<Long>, NodeInfo> _iter204 : this.executor_node_port.entrySet())
+          {
+            {
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter204.getKey().size()));
+              for (long _iter205 : _iter204.getKey())
+              {
+                oprot.writeI64(_iter205);
+              }
+              oprot.writeListEnd();
+            }
+            _iter204.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.executor_start_time_secs != null) {
+      if (is_set_executor_start_time_secs()) {
+        oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, this.executor_start_time_secs.size()));
+          for (Map.Entry<List<Long>, Long> _iter206 : this.executor_start_time_secs.entrySet())
+          {
+            {
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter206.getKey().size()));
+              for (long _iter207 : _iter206.getKey())
+              {
+                oprot.writeI64(_iter207);
+              }
+              oprot.writeListEnd();
+            }
+            oprot.writeI64(_iter206.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Assignment(");
+    boolean first = true;
+
+    sb.append("master_code_dir:");
+    if (this.master_code_dir == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.master_code_dir);
+    }
+    first = false;
+    if (is_set_node_host()) {
+      if (!first) sb.append(", ");
+      sb.append("node_host:");
+      if (this.node_host == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.node_host);
+      }
+      first = false;
+    }
+    if (is_set_executor_node_port()) {
+      if (!first) sb.append(", ");
+      sb.append("executor_node_port:");
+      if (this.executor_node_port == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.executor_node_port);
+      }
+      first = false;
+    }
+    if (is_set_executor_start_time_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("executor_start_time_secs:");
+      if (this.executor_start_time_secs == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.executor_start_time_secs);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_master_code_dir()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'master_code_dir' is unset! Struct:" + toString());
+    }
+
+  }
+
+  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);
+    }
+  }
+
+}
+


[12/16] storm git commit: Deleting accidently checked in file. Renaming ZKWorkerHeartbeat to ClusterWorkerHeartbeat.

Posted by bo...@apache.org.
Deleting accidently checked in file. Renaming ZKWorkerHeartbeat to ClusterWorkerHeartbeat.


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

Branch: refs/heads/master
Commit: 5a66bb6d2ffbae8c1b7ea929dcb256413bee5b50
Parents: 539a6a2
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Mar 11 13:04:31 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Mar 11 13:04:31 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   |   4 +-
 storm-core/src/clj/backtype/storm/converter.clj |   6 +-
 .../storm/generated/ClusterWorkerHeartbeat.java | 673 +++++++++++++++++++
 .../storm/generated/ZKWorkerHeartbeat.java      | 673 -------------------
 .../jvm/backtype/storm/serialization/Test.java  |  17 -
 storm-core/src/py/storm/ttypes.py               |   4 +-
 storm-core/src/storm.thrift                     |   3 +-
 7 files changed, 682 insertions(+), 698 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5a66bb6d/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index 8866d04..7987a30 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -16,7 +16,7 @@
 
 (ns backtype.storm.cluster
   (:import [org.apache.zookeeper.data Stat ACL Id]
-           [backtype.storm.generated SupervisorInfo Assignment StormBase ZKWorkerHeartbeat ErrorInfo Credentials]
+           [backtype.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials]
            [java.io Serializable])
   (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
   (:import [backtype.storm.utils Utils])
@@ -339,7 +339,7 @@
         (let [worker-hb (get-data cluster-state (workerbeat-path storm-id node port) false)]
           (if worker-hb
             (-> worker-hb
-              (maybe-deserialize ZKWorkerHeartbeat)
+              (maybe-deserialize ClusterWorkerHeartbeat)
               clojurify-zk-worker-hb))))
 
 

http://git-wip-us.apache.org/repos/asf/storm/blob/5a66bb6d/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 e011798..6102ced 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -1,6 +1,6 @@
 (ns backtype.storm.converter
   (:import [backtype.storm.generated SupervisorInfo NodeInfo Assignment
-            StormBase TopologyStatus ZKWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions TopologyActionOptions])
+            StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions TopologyActionOptions])
   (:use [backtype.storm util stats log])
   (:require [backtype.storm.daemon [common :as common]]))
 
@@ -158,7 +158,7 @@
         stats))
     {}))
 
-(defn clojurify-zk-worker-hb [^ZKWorkerHeartbeat worker-hb]
+(defn clojurify-zk-worker-hb [^ClusterWorkerHeartbeat worker-hb]
   (if worker-hb
     {:storm-id (.get_storm_id worker-hb)
      :executor-stats (clojurify-stats (into {} (.get_executor_stats worker-hb)))
@@ -169,7 +169,7 @@
 
 (defn thriftify-zk-worker-hb [worker-hb]
   (if (not-empty (filter second (:executor-stats worker-hb)))
-    (doto (ZKWorkerHeartbeat.)
+    (doto (ClusterWorkerHeartbeat.)
       (.set_storm_id (:storm-id worker-hb))
       (.set_executor_stats (thriftify-stats (filter second (:executor-stats worker-hb))))
       (.set_time_secs (:time-secs worker-hb)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/5a66bb6d/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
new file mode 100644
index 0000000..fb04e3a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
@@ -0,0 +1,673 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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-3-11")
+public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWorkerHeartbeat, ClusterWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterWorkerHeartbeat> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterWorkerHeartbeat");
+
+  private static final org.apache.thrift.protocol.TField STORM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storm_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_stats", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ClusterWorkerHeartbeatStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ClusterWorkerHeartbeatTupleSchemeFactory());
+  }
+
+  private String storm_id; // required
+  private Map<ExecutorInfo,ExecutorStats> executor_stats; // required
+  private int time_secs; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    STORM_ID((short)1, "storm_id"),
+    EXECUTOR_STATS((short)2, "executor_stats"),
+    TIME_SECS((short)3, "time_secs");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STORM_ID
+          return STORM_ID;
+        case 2: // EXECUTOR_STATS
+          return EXECUTOR_STATS;
+        case 3: // TIME_SECS
+          return TIME_SECS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_SECS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STORM_ID, new org.apache.thrift.meta_data.FieldMetaData("storm_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EXECUTOR_STATS, new org.apache.thrift.meta_data.FieldMetaData("executor_stats", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorStats.class))));
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClusterWorkerHeartbeat.class, metaDataMap);
+  }
+
+  public ClusterWorkerHeartbeat() {
+  }
+
+  public ClusterWorkerHeartbeat(
+    String storm_id,
+    Map<ExecutorInfo,ExecutorStats> executor_stats,
+    int time_secs)
+  {
+    this();
+    this.storm_id = storm_id;
+    this.executor_stats = executor_stats;
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ClusterWorkerHeartbeat(ClusterWorkerHeartbeat other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_storm_id()) {
+      this.storm_id = other.storm_id;
+    }
+    if (other.is_set_executor_stats()) {
+      Map<ExecutorInfo,ExecutorStats> __this__executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(other.executor_stats.size());
+      for (Map.Entry<ExecutorInfo, ExecutorStats> other_element : other.executor_stats.entrySet()) {
+
+        ExecutorInfo other_element_key = other_element.getKey();
+        ExecutorStats other_element_value = other_element.getValue();
+
+        ExecutorInfo __this__executor_stats_copy_key = new ExecutorInfo(other_element_key);
+
+        ExecutorStats __this__executor_stats_copy_value = new ExecutorStats(other_element_value);
+
+        __this__executor_stats.put(__this__executor_stats_copy_key, __this__executor_stats_copy_value);
+      }
+      this.executor_stats = __this__executor_stats;
+    }
+    this.time_secs = other.time_secs;
+  }
+
+  public ClusterWorkerHeartbeat deepCopy() {
+    return new ClusterWorkerHeartbeat(this);
+  }
+
+  @Override
+  public void clear() {
+    this.storm_id = null;
+    this.executor_stats = null;
+    set_time_secs_isSet(false);
+    this.time_secs = 0;
+  }
+
+  public String get_storm_id() {
+    return this.storm_id;
+  }
+
+  public void set_storm_id(String storm_id) {
+    this.storm_id = storm_id;
+  }
+
+  public void unset_storm_id() {
+    this.storm_id = null;
+  }
+
+  /** Returns true if field storm_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_storm_id() {
+    return this.storm_id != null;
+  }
+
+  public void set_storm_id_isSet(boolean value) {
+    if (!value) {
+      this.storm_id = null;
+    }
+  }
+
+  public int get_executor_stats_size() {
+    return (this.executor_stats == null) ? 0 : this.executor_stats.size();
+  }
+
+  public void put_to_executor_stats(ExecutorInfo key, ExecutorStats val) {
+    if (this.executor_stats == null) {
+      this.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>();
+    }
+    this.executor_stats.put(key, val);
+  }
+
+  public Map<ExecutorInfo,ExecutorStats> get_executor_stats() {
+    return this.executor_stats;
+  }
+
+  public void set_executor_stats(Map<ExecutorInfo,ExecutorStats> executor_stats) {
+    this.executor_stats = executor_stats;
+  }
+
+  public void unset_executor_stats() {
+    this.executor_stats = null;
+  }
+
+  /** Returns true if field executor_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executor_stats() {
+    return this.executor_stats != null;
+  }
+
+  public void set_executor_stats_isSet(boolean value) {
+    if (!value) {
+      this.executor_stats = null;
+    }
+  }
+
+  public int get_time_secs() {
+    return this.time_secs;
+  }
+
+  public void set_time_secs(int time_secs) {
+    this.time_secs = time_secs;
+    set_time_secs_isSet(true);
+  }
+
+  public void unset_time_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
+  }
+
+  public void set_time_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STORM_ID:
+      if (value == null) {
+        unset_storm_id();
+      } else {
+        set_storm_id((String)value);
+      }
+      break;
+
+    case EXECUTOR_STATS:
+      if (value == null) {
+        unset_executor_stats();
+      } else {
+        set_executor_stats((Map<ExecutorInfo,ExecutorStats>)value);
+      }
+      break;
+
+    case TIME_SECS:
+      if (value == null) {
+        unset_time_secs();
+      } else {
+        set_time_secs((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STORM_ID:
+      return get_storm_id();
+
+    case EXECUTOR_STATS:
+      return get_executor_stats();
+
+    case TIME_SECS:
+      return Integer.valueOf(get_time_secs());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case STORM_ID:
+      return is_set_storm_id();
+    case EXECUTOR_STATS:
+      return is_set_executor_stats();
+    case TIME_SECS:
+      return is_set_time_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ClusterWorkerHeartbeat)
+      return this.equals((ClusterWorkerHeartbeat)that);
+    return false;
+  }
+
+  public boolean equals(ClusterWorkerHeartbeat that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_storm_id = true && this.is_set_storm_id();
+    boolean that_present_storm_id = true && that.is_set_storm_id();
+    if (this_present_storm_id || that_present_storm_id) {
+      if (!(this_present_storm_id && that_present_storm_id))
+        return false;
+      if (!this.storm_id.equals(that.storm_id))
+        return false;
+    }
+
+    boolean this_present_executor_stats = true && this.is_set_executor_stats();
+    boolean that_present_executor_stats = true && that.is_set_executor_stats();
+    if (this_present_executor_stats || that_present_executor_stats) {
+      if (!(this_present_executor_stats && that_present_executor_stats))
+        return false;
+      if (!this.executor_stats.equals(that.executor_stats))
+        return false;
+    }
+
+    boolean this_present_time_secs = true;
+    boolean that_present_time_secs = true;
+    if (this_present_time_secs || that_present_time_secs) {
+      if (!(this_present_time_secs && that_present_time_secs))
+        return false;
+      if (this.time_secs != that.time_secs)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_storm_id = true && (is_set_storm_id());
+    list.add(present_storm_id);
+    if (present_storm_id)
+      list.add(storm_id);
+
+    boolean present_executor_stats = true && (is_set_executor_stats());
+    list.add(present_executor_stats);
+    if (present_executor_stats)
+      list.add(executor_stats);
+
+    boolean present_time_secs = true;
+    list.add(present_time_secs);
+    if (present_time_secs)
+      list.add(time_secs);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ClusterWorkerHeartbeat other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_storm_id()).compareTo(other.is_set_storm_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_storm_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_id, other.storm_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executor_stats()).compareTo(other.is_set_executor_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executor_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_stats, other.executor_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ClusterWorkerHeartbeat(");
+    boolean first = true;
+
+    sb.append("storm_id:");
+    if (this.storm_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.storm_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executor_stats:");
+    if (this.executor_stats == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executor_stats);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("time_secs:");
+    sb.append(this.time_secs);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_storm_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'storm_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executor_stats()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executor_stats' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_time_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ClusterWorkerHeartbeatStandardSchemeFactory implements SchemeFactory {
+    public ClusterWorkerHeartbeatStandardScheme getScheme() {
+      return new ClusterWorkerHeartbeatStandardScheme();
+    }
+  }
+
+  private static class ClusterWorkerHeartbeatStandardScheme extends StandardScheme<ClusterWorkerHeartbeat> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // STORM_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.storm_id = iprot.readString();
+              struct.set_storm_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // EXECUTOR_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map426.size);
+                ExecutorInfo _key427;
+                ExecutorStats _val428;
+                for (int _i429 = 0; _i429 < _map426.size; ++_i429)
+                {
+                  _key427 = new ExecutorInfo();
+                  _key427.read(iprot);
+                  _val428 = new ExecutorStats();
+                  _val428.read(iprot);
+                  struct.executor_stats.put(_key427, _val428);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executor_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.time_secs = iprot.readI32();
+              struct.set_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.storm_id != null) {
+        oprot.writeFieldBegin(STORM_ID_FIELD_DESC);
+        oprot.writeString(struct.storm_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.executor_stats != null) {
+        oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter430 : struct.executor_stats.entrySet())
+          {
+            _iter430.getKey().write(oprot);
+            _iter430.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.time_secs);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ClusterWorkerHeartbeatTupleSchemeFactory implements SchemeFactory {
+    public ClusterWorkerHeartbeatTupleScheme getScheme() {
+      return new ClusterWorkerHeartbeatTupleScheme();
+    }
+  }
+
+  private static class ClusterWorkerHeartbeatTupleScheme extends TupleScheme<ClusterWorkerHeartbeat> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.storm_id);
+      {
+        oprot.writeI32(struct.executor_stats.size());
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter431 : struct.executor_stats.entrySet())
+        {
+          _iter431.getKey().write(oprot);
+          _iter431.getValue().write(oprot);
+        }
+      }
+      oprot.writeI32(struct.time_secs);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.storm_id = iprot.readString();
+      struct.set_storm_id_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map432 = 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*_map432.size);
+        ExecutorInfo _key433;
+        ExecutorStats _val434;
+        for (int _i435 = 0; _i435 < _map432.size; ++_i435)
+        {
+          _key433 = new ExecutorInfo();
+          _key433.read(iprot);
+          _val434 = new ExecutorStats();
+          _val434.read(iprot);
+          struct.executor_stats.put(_key433, _val434);
+        }
+      }
+      struct.set_executor_stats_isSet(true);
+      struct.time_secs = iprot.readI32();
+      struct.set_time_secs_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/5a66bb6d/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
deleted file mode 100644
index 01c5c67..0000000
--- a/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
+++ /dev/null
@@ -1,673 +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-3-5")
-public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartbeat, ZKWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ZKWorkerHeartbeat> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ZKWorkerHeartbeat");
-
-  private static final org.apache.thrift.protocol.TField STORM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storm_id", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField EXECUTOR_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_stats", org.apache.thrift.protocol.TType.MAP, (short)2);
-  private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)3);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new ZKWorkerHeartbeatStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new ZKWorkerHeartbeatTupleSchemeFactory());
-  }
-
-  private String storm_id; // required
-  private Map<ExecutorInfo,ExecutorStats> executor_stats; // required
-  private int time_secs; // required
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    STORM_ID((short)1, "storm_id"),
-    EXECUTOR_STATS((short)2, "executor_stats"),
-    TIME_SECS((short)3, "time_secs");
-
-    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-    static {
-      for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byName.put(field.getFieldName(), field);
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, or null if its not found.
-     */
-    public static _Fields findByThriftId(int fieldId) {
-      switch(fieldId) {
-        case 1: // STORM_ID
-          return STORM_ID;
-        case 2: // EXECUTOR_STATS
-          return EXECUTOR_STATS;
-        case 3: // TIME_SECS
-          return TIME_SECS;
-        default:
-          return null;
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, throwing an exception
-     * if it is not found.
-     */
-    public static _Fields findByThriftIdOrThrow(int fieldId) {
-      _Fields fields = findByThriftId(fieldId);
-      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-      return fields;
-    }
-
-    /**
-     * Find the _Fields constant that matches name, or null if its not found.
-     */
-    public static _Fields findByName(String name) {
-      return byName.get(name);
-    }
-
-    private final short _thriftId;
-    private final String _fieldName;
-
-    _Fields(short thriftId, String fieldName) {
-      _thriftId = thriftId;
-      _fieldName = fieldName;
-    }
-
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    public String getFieldName() {
-      return _fieldName;
-    }
-  }
-
-  // isset id assignments
-  private static final int __TIME_SECS_ISSET_ID = 0;
-  private byte __isset_bitfield = 0;
-  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-  static {
-    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.STORM_ID, new org.apache.thrift.meta_data.FieldMetaData("storm_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.EXECUTOR_STATS, new org.apache.thrift.meta_data.FieldMetaData("executor_stats", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class), 
-            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorStats.class))));
-    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ZKWorkerHeartbeat.class, metaDataMap);
-  }
-
-  public ZKWorkerHeartbeat() {
-  }
-
-  public ZKWorkerHeartbeat(
-    String storm_id,
-    Map<ExecutorInfo,ExecutorStats> executor_stats,
-    int time_secs)
-  {
-    this();
-    this.storm_id = storm_id;
-    this.executor_stats = executor_stats;
-    this.time_secs = time_secs;
-    set_time_secs_isSet(true);
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public ZKWorkerHeartbeat(ZKWorkerHeartbeat other) {
-    __isset_bitfield = other.__isset_bitfield;
-    if (other.is_set_storm_id()) {
-      this.storm_id = other.storm_id;
-    }
-    if (other.is_set_executor_stats()) {
-      Map<ExecutorInfo,ExecutorStats> __this__executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(other.executor_stats.size());
-      for (Map.Entry<ExecutorInfo, ExecutorStats> other_element : other.executor_stats.entrySet()) {
-
-        ExecutorInfo other_element_key = other_element.getKey();
-        ExecutorStats other_element_value = other_element.getValue();
-
-        ExecutorInfo __this__executor_stats_copy_key = new ExecutorInfo(other_element_key);
-
-        ExecutorStats __this__executor_stats_copy_value = new ExecutorStats(other_element_value);
-
-        __this__executor_stats.put(__this__executor_stats_copy_key, __this__executor_stats_copy_value);
-      }
-      this.executor_stats = __this__executor_stats;
-    }
-    this.time_secs = other.time_secs;
-  }
-
-  public ZKWorkerHeartbeat deepCopy() {
-    return new ZKWorkerHeartbeat(this);
-  }
-
-  @Override
-  public void clear() {
-    this.storm_id = null;
-    this.executor_stats = null;
-    set_time_secs_isSet(false);
-    this.time_secs = 0;
-  }
-
-  public String get_storm_id() {
-    return this.storm_id;
-  }
-
-  public void set_storm_id(String storm_id) {
-    this.storm_id = storm_id;
-  }
-
-  public void unset_storm_id() {
-    this.storm_id = null;
-  }
-
-  /** Returns true if field storm_id is set (has been assigned a value) and false otherwise */
-  public boolean is_set_storm_id() {
-    return this.storm_id != null;
-  }
-
-  public void set_storm_id_isSet(boolean value) {
-    if (!value) {
-      this.storm_id = null;
-    }
-  }
-
-  public int get_executor_stats_size() {
-    return (this.executor_stats == null) ? 0 : this.executor_stats.size();
-  }
-
-  public void put_to_executor_stats(ExecutorInfo key, ExecutorStats val) {
-    if (this.executor_stats == null) {
-      this.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>();
-    }
-    this.executor_stats.put(key, val);
-  }
-
-  public Map<ExecutorInfo,ExecutorStats> get_executor_stats() {
-    return this.executor_stats;
-  }
-
-  public void set_executor_stats(Map<ExecutorInfo,ExecutorStats> executor_stats) {
-    this.executor_stats = executor_stats;
-  }
-
-  public void unset_executor_stats() {
-    this.executor_stats = null;
-  }
-
-  /** Returns true if field executor_stats is set (has been assigned a value) and false otherwise */
-  public boolean is_set_executor_stats() {
-    return this.executor_stats != null;
-  }
-
-  public void set_executor_stats_isSet(boolean value) {
-    if (!value) {
-      this.executor_stats = null;
-    }
-  }
-
-  public int get_time_secs() {
-    return this.time_secs;
-  }
-
-  public void set_time_secs(int time_secs) {
-    this.time_secs = time_secs;
-    set_time_secs_isSet(true);
-  }
-
-  public void unset_time_secs() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
-  }
-
-  /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
-  public boolean is_set_time_secs() {
-    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
-  }
-
-  public void set_time_secs_isSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case STORM_ID:
-      if (value == null) {
-        unset_storm_id();
-      } else {
-        set_storm_id((String)value);
-      }
-      break;
-
-    case EXECUTOR_STATS:
-      if (value == null) {
-        unset_executor_stats();
-      } else {
-        set_executor_stats((Map<ExecutorInfo,ExecutorStats>)value);
-      }
-      break;
-
-    case TIME_SECS:
-      if (value == null) {
-        unset_time_secs();
-      } else {
-        set_time_secs((Integer)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case STORM_ID:
-      return get_storm_id();
-
-    case EXECUTOR_STATS:
-      return get_executor_stats();
-
-    case TIME_SECS:
-      return Integer.valueOf(get_time_secs());
-
-    }
-    throw new IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new IllegalArgumentException();
-    }
-
-    switch (field) {
-    case STORM_ID:
-      return is_set_storm_id();
-    case EXECUTOR_STATS:
-      return is_set_executor_stats();
-    case TIME_SECS:
-      return is_set_time_secs();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof ZKWorkerHeartbeat)
-      return this.equals((ZKWorkerHeartbeat)that);
-    return false;
-  }
-
-  public boolean equals(ZKWorkerHeartbeat that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_storm_id = true && this.is_set_storm_id();
-    boolean that_present_storm_id = true && that.is_set_storm_id();
-    if (this_present_storm_id || that_present_storm_id) {
-      if (!(this_present_storm_id && that_present_storm_id))
-        return false;
-      if (!this.storm_id.equals(that.storm_id))
-        return false;
-    }
-
-    boolean this_present_executor_stats = true && this.is_set_executor_stats();
-    boolean that_present_executor_stats = true && that.is_set_executor_stats();
-    if (this_present_executor_stats || that_present_executor_stats) {
-      if (!(this_present_executor_stats && that_present_executor_stats))
-        return false;
-      if (!this.executor_stats.equals(that.executor_stats))
-        return false;
-    }
-
-    boolean this_present_time_secs = true;
-    boolean that_present_time_secs = true;
-    if (this_present_time_secs || that_present_time_secs) {
-      if (!(this_present_time_secs && that_present_time_secs))
-        return false;
-      if (this.time_secs != that.time_secs)
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    List<Object> list = new ArrayList<Object>();
-
-    boolean present_storm_id = true && (is_set_storm_id());
-    list.add(present_storm_id);
-    if (present_storm_id)
-      list.add(storm_id);
-
-    boolean present_executor_stats = true && (is_set_executor_stats());
-    list.add(present_executor_stats);
-    if (present_executor_stats)
-      list.add(executor_stats);
-
-    boolean present_time_secs = true;
-    list.add(present_time_secs);
-    if (present_time_secs)
-      list.add(time_secs);
-
-    return list.hashCode();
-  }
-
-  @Override
-  public int compareTo(ZKWorkerHeartbeat other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = Boolean.valueOf(is_set_storm_id()).compareTo(other.is_set_storm_id());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (is_set_storm_id()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_id, other.storm_id);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(is_set_executor_stats()).compareTo(other.is_set_executor_stats());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (is_set_executor_stats()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_stats, other.executor_stats);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (is_set_time_secs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    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("ZKWorkerHeartbeat(");
-    boolean first = true;
-
-    sb.append("storm_id:");
-    if (this.storm_id == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.storm_id);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("executor_stats:");
-    if (this.executor_stats == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.executor_stats);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("time_secs:");
-    sb.append(this.time_secs);
-    first = false;
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    if (!is_set_storm_id()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'storm_id' is unset! Struct:" + toString());
-    }
-
-    if (!is_set_executor_stats()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executor_stats' is unset! Struct:" + toString());
-    }
-
-    if (!is_set_time_secs()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
-    }
-
-    // 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 ZKWorkerHeartbeatStandardSchemeFactory implements SchemeFactory {
-    public ZKWorkerHeartbeatStandardScheme getScheme() {
-      return new ZKWorkerHeartbeatStandardScheme();
-    }
-  }
-
-  private static class ZKWorkerHeartbeatStandardScheme extends StandardScheme<ZKWorkerHeartbeat> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField schemeField;
-      iprot.readStructBegin();
-      while (true)
-      {
-        schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (schemeField.id) {
-          case 1: // STORM_ID
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.storm_id = iprot.readString();
-              struct.set_storm_id_isSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // EXECUTOR_STATS
-            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-              {
-                org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
-                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map426.size);
-                ExecutorInfo _key427;
-                ExecutorStats _val428;
-                for (int _i429 = 0; _i429 < _map426.size; ++_i429)
-                {
-                  _key427 = new ExecutorInfo();
-                  _key427.read(iprot);
-                  _val428 = new ExecutorStats();
-                  _val428.read(iprot);
-                  struct.executor_stats.put(_key427, _val428);
-                }
-                iprot.readMapEnd();
-              }
-              struct.set_executor_stats_isSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // TIME_SECS
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.time_secs = iprot.readI32();
-              struct.set_time_secs_isSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          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, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.storm_id != null) {
-        oprot.writeFieldBegin(STORM_ID_FIELD_DESC);
-        oprot.writeString(struct.storm_id);
-        oprot.writeFieldEnd();
-      }
-      if (struct.executor_stats != null) {
-        oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
-        {
-          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
-          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter430 : struct.executor_stats.entrySet())
-          {
-            _iter430.getKey().write(oprot);
-            _iter430.getValue().write(oprot);
-          }
-          oprot.writeMapEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
-      oprot.writeI32(struct.time_secs);
-      oprot.writeFieldEnd();
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class ZKWorkerHeartbeatTupleSchemeFactory implements SchemeFactory {
-    public ZKWorkerHeartbeatTupleScheme getScheme() {
-      return new ZKWorkerHeartbeatTupleScheme();
-    }
-  }
-
-  private static class ZKWorkerHeartbeatTupleScheme extends TupleScheme<ZKWorkerHeartbeat> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      oprot.writeString(struct.storm_id);
-      {
-        oprot.writeI32(struct.executor_stats.size());
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter431 : struct.executor_stats.entrySet())
-        {
-          _iter431.getKey().write(oprot);
-          _iter431.getValue().write(oprot);
-        }
-      }
-      oprot.writeI32(struct.time_secs);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      struct.storm_id = iprot.readString();
-      struct.set_storm_id_isSet(true);
-      {
-        org.apache.thrift.protocol.TMap _map432 = 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*_map432.size);
-        ExecutorInfo _key433;
-        ExecutorStats _val434;
-        for (int _i435 = 0; _i435 < _map432.size; ++_i435)
-        {
-          _key433 = new ExecutorInfo();
-          _key433.read(iprot);
-          _val434 = new ExecutorStats();
-          _val434.read(iprot);
-          struct.executor_stats.put(_key433, _val434);
-        }
-      }
-      struct.set_executor_stats_isSet(true);
-      struct.time_secs = iprot.readI32();
-      struct.set_time_secs_isSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/5a66bb6d/storm-core/src/jvm/backtype/storm/serialization/Test.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/Test.java b/storm-core/src/jvm/backtype/storm/serialization/Test.java
deleted file mode 100644
index 02b2bab..0000000
--- a/storm-core/src/jvm/backtype/storm/serialization/Test.java
+++ /dev/null
@@ -1,17 +0,0 @@
-package backtype.storm.serialization;
-
-import backtype.storm.generated.SupervisorInfo;
-
-/**
- * Created by pbrahmbhatt on 1/29/15.
- */
-public class Test {
-
-    public static void main(String[] args) throws Exception {
-//        ThriftSerializationDelegate t = new ThriftSerializationDelegate();
-//        t.serialize(null);
-
-        DefaultSerializationDelegate d = new DefaultSerializationDelegate();
-        System.out.println(d.serialize(null));
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/5a66bb6d/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 924f761..f683dfe 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -5249,7 +5249,7 @@ class StormBase:
   def __ne__(self, other):
     return not (self == other)
 
-class ZKWorkerHeartbeat:
+class ClusterWorkerHeartbeat:
   """
   Attributes:
    - storm_id
@@ -5310,7 +5310,7 @@ class ZKWorkerHeartbeat:
     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('ZKWorkerHeartbeat')
+    oprot.writeStructBegin('ClusterWorkerHeartbeat')
     if self.storm_id is not None:
       oprot.writeFieldBegin('storm_id', TType.STRING, 1)
       oprot.writeString(self.storm_id.encode('utf-8'))

http://git-wip-us.apache.org/repos/asf/storm/blob/5a66bb6d/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 04b6a1b..db4a7b3 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -288,11 +288,12 @@ struct StormBase {
     8: optional TopologyStatus prev_status;//currently only used during rebalance action.
 }
 
-struct ZKWorkerHeartbeat {
+struct ClusterWorkerHeartbeat {
     1: required string storm_id;
     2: required map<ExecutorInfo,ExecutorStats> executor_stats;
     3: required i32 time_secs;
 }
+
 enum NumErrorsChoice {
   ALL,
   NONE,


[10/16] storm git commit: Regenrated the files using thrift-0.9.2 and fixed some bugs introduced by upmerging.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/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 e27d635..b00f072 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StormBase.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
@@ -16,13 +16,24 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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.commons.lang.builder.HashCodeBuilder;
+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;
@@ -35,10 +46,13 @@ 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;
 
-public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable {
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
 
   private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
@@ -50,14 +64,20 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   private static final org.apache.thrift.protocol.TField TOPOLOGY_ACTION_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_action_options", org.apache.thrift.protocol.TType.STRUCT, (short)7);
   private static final org.apache.thrift.protocol.TField PREV_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("prev_status", org.apache.thrift.protocol.TType.I32, (short)8);
 
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new StormBaseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new StormBaseTupleSchemeFactory());
+  }
+
   private String name; // required
   private TopologyStatus status; // required
   private int num_workers; // required
-  private Map<String,Integer> component_executors; // required
-  private int launch_time_secs; // required
-  private String owner; // required
-  private TopologyActionOptions topology_action_options; // required
-  private TopologyStatus prev_status; // required
+  private Map<String,Integer> component_executors; // optional
+  private int launch_time_secs; // optional
+  private String owner; // optional
+  private TopologyActionOptions topology_action_options; // optional
+  private TopologyStatus prev_status; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -149,28 +169,28 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   // isset id assignments
   private static final int __NUM_WORKERS_ISSET_ID = 0;
   private static final int __LAUNCH_TIME_SECS_ISSET_ID = 1;
-  private BitSet __isset_bit_vector = new BitSet(2);
-
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.COMPONENT_EXECUTORS,_Fields.LAUNCH_TIME_SECS,_Fields.OWNER,_Fields.TOPOLOGY_ACTION_OPTIONS,_Fields.PREV_STATUS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
-    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.COMPONENT_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("component_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL,
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+    tmpMap.put(_Fields.COMPONENT_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("component_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
-    tmpMap.put(_Fields.LAUNCH_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("launch_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.LAUNCH_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("launch_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.TOPOLOGY_ACTION_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("topology_action_options", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.TOPOLOGY_ACTION_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("topology_action_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyActionOptions.class)));
-    tmpMap.put(_Fields.PREV_STATUS, new org.apache.thrift.meta_data.FieldMetaData("prev_status", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.PREV_STATUS, new org.apache.thrift.meta_data.FieldMetaData("prev_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormBase.class, metaDataMap);
@@ -195,8 +215,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
    * Performs a deep copy on <i>other</i>.
    */
   public StormBase(StormBase other) {
-    __isset_bit_vector.clear();
-    __isset_bit_vector.or(other.__isset_bit_vector);
+    __isset_bitfield = other.__isset_bitfield;
     if (other.is_set_name()) {
       this.name = other.name;
     }
@@ -205,18 +224,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     }
     this.num_workers = other.num_workers;
     if (other.is_set_component_executors()) {
-      Map<String,Integer> __this__component_executors = new HashMap<String,Integer>();
-      for (Map.Entry<String, Integer> other_element : other.component_executors.entrySet()) {
-
-        String other_element_key = other_element.getKey();
-        Integer other_element_value = other_element.getValue();
-
-        String __this__component_executors_copy_key = other_element_key;
-
-        Integer __this__component_executors_copy_value = other_element_value;
-
-        __this__component_executors.put(__this__component_executors_copy_key, __this__component_executors_copy_value);
-      }
+      Map<String,Integer> __this__component_executors = new HashMap<String,Integer>(other.component_executors);
       this.component_executors = __this__component_executors;
     }
     this.launch_time_secs = other.launch_time_secs;
@@ -313,16 +321,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   }
 
   public void unset_num_workers() {
-    __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID);
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
   }
 
   /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
   public boolean is_set_num_workers() {
-    return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID);
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
   }
 
   public void set_num_workers_isSet(boolean value) {
-    __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value);
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value);
   }
 
   public int get_component_executors_size() {
@@ -369,16 +377,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   }
 
   public void unset_launch_time_secs() {
-    __isset_bit_vector.clear(__LAUNCH_TIME_SECS_ISSET_ID);
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LAUNCH_TIME_SECS_ISSET_ID);
   }
 
   /** Returns true if field launch_time_secs is set (has been assigned a value) and false otherwise */
   public boolean is_set_launch_time_secs() {
-    return __isset_bit_vector.get(__LAUNCH_TIME_SECS_ISSET_ID);
+    return EncodingUtils.testBit(__isset_bitfield, __LAUNCH_TIME_SECS_ISSET_ID);
   }
 
   public void set_launch_time_secs_isSet(boolean value) {
-    __isset_bit_vector.set(__LAUNCH_TIME_SECS_ISSET_ID, value);
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LAUNCH_TIME_SECS_ISSET_ID, value);
   }
 
   public String get_owner() {
@@ -674,135 +682,135 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
+    List<Object> list = new ArrayList<Object>();
 
     boolean present_name = true && (is_set_name());
-    builder.append(present_name);
+    list.add(present_name);
     if (present_name)
-      builder.append(name);
+      list.add(name);
 
     boolean present_status = true && (is_set_status());
-    builder.append(present_status);
+    list.add(present_status);
     if (present_status)
-      builder.append(status.getValue());
+      list.add(status.getValue());
 
     boolean present_num_workers = true;
-    builder.append(present_num_workers);
+    list.add(present_num_workers);
     if (present_num_workers)
-      builder.append(num_workers);
+      list.add(num_workers);
 
     boolean present_component_executors = true && (is_set_component_executors());
-    builder.append(present_component_executors);
+    list.add(present_component_executors);
     if (present_component_executors)
-      builder.append(component_executors);
+      list.add(component_executors);
 
     boolean present_launch_time_secs = true && (is_set_launch_time_secs());
-    builder.append(present_launch_time_secs);
+    list.add(present_launch_time_secs);
     if (present_launch_time_secs)
-      builder.append(launch_time_secs);
+      list.add(launch_time_secs);
 
     boolean present_owner = true && (is_set_owner());
-    builder.append(present_owner);
+    list.add(present_owner);
     if (present_owner)
-      builder.append(owner);
+      list.add(owner);
 
     boolean present_topology_action_options = true && (is_set_topology_action_options());
-    builder.append(present_topology_action_options);
+    list.add(present_topology_action_options);
     if (present_topology_action_options)
-      builder.append(topology_action_options);
+      list.add(topology_action_options);
 
     boolean present_prev_status = true && (is_set_prev_status());
-    builder.append(present_prev_status);
+    list.add(present_prev_status);
     if (present_prev_status)
-      builder.append(prev_status.getValue());
+      list.add(prev_status.getValue());
 
-    return builder.toHashCode();
+    return list.hashCode();
   }
 
+  @Override
   public int compareTo(StormBase other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
 
     int lastComparison = 0;
-    StormBase typedOther = (StormBase)other;
 
-    lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name());
+    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, typedOther.name);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_status()).compareTo(typedOther.is_set_status());
+    lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_status()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers());
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_num_workers()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_component_executors()).compareTo(typedOther.is_set_component_executors());
+    lastComparison = Boolean.valueOf(is_set_component_executors()).compareTo(other.is_set_component_executors());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_component_executors()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_executors, typedOther.component_executors);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_executors, other.component_executors);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_launch_time_secs()).compareTo(typedOther.is_set_launch_time_secs());
+    lastComparison = Boolean.valueOf(is_set_launch_time_secs()).compareTo(other.is_set_launch_time_secs());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_launch_time_secs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.launch_time_secs, typedOther.launch_time_secs);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.launch_time_secs, other.launch_time_secs);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(typedOther.is_set_owner());
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_owner()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_topology_action_options()).compareTo(typedOther.is_set_topology_action_options());
+    lastComparison = Boolean.valueOf(is_set_topology_action_options()).compareTo(other.is_set_topology_action_options());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_topology_action_options()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_action_options, typedOther.topology_action_options);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_action_options, other.topology_action_options);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_prev_status()).compareTo(typedOther.is_set_prev_status());
+    lastComparison = Boolean.valueOf(is_set_prev_status()).compareTo(other.is_set_prev_status());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_prev_status()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.prev_status, typedOther.prev_status);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.prev_status, other.prev_status);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -815,155 +823,11 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) {
-        break;
-      }
-      switch (field.id) {
-        case 1: // NAME
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.name = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // STATUS
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.status = TopologyStatus.findByValue(iprot.readI32());
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 3: // NUM_WORKERS
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.num_workers = iprot.readI32();
-            set_num_workers_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 4: // COMPONENT_EXECUTORS
-          if (field.type == org.apache.thrift.protocol.TType.MAP) {
-            {
-              org.apache.thrift.protocol.TMap _map208 = iprot.readMapBegin();
-              this.component_executors = new HashMap<String,Integer>(2*_map208.size);
-              for (int _i209 = 0; _i209 < _map208.size; ++_i209)
-              {
-                String _key210; // required
-                int _val211; // required
-                _key210 = iprot.readString();
-                _val211 = iprot.readI32();
-                this.component_executors.put(_key210, _val211);
-              }
-              iprot.readMapEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 5: // LAUNCH_TIME_SECS
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.launch_time_secs = iprot.readI32();
-            set_launch_time_secs_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 6: // OWNER
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.owner = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 7: // TOPOLOGY_ACTION_OPTIONS
-          if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
-            this.topology_action_options = new TopologyActionOptions();
-            this.topology_action_options.read(iprot);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 8: // PREV_STATUS
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.prev_status = TopologyStatus.findByValue(iprot.readI32());
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    if (this.name != null) {
-      oprot.writeFieldBegin(NAME_FIELD_DESC);
-      oprot.writeString(this.name);
-      oprot.writeFieldEnd();
-    }
-    if (this.status != null) {
-      oprot.writeFieldBegin(STATUS_FIELD_DESC);
-      oprot.writeI32(this.status.getValue());
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
-    oprot.writeI32(this.num_workers);
-    oprot.writeFieldEnd();
-    if (this.component_executors != null) {
-      if (is_set_component_executors()) {
-        oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
-        {
-          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, this.component_executors.size()));
-          for (Map.Entry<String, Integer> _iter212 : this.component_executors.entrySet())
-          {
-            oprot.writeString(_iter212.getKey());
-            oprot.writeI32(_iter212.getValue());
-          }
-          oprot.writeMapEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-    }
-    if (is_set_launch_time_secs()) {
-      oprot.writeFieldBegin(LAUNCH_TIME_SECS_FIELD_DESC);
-      oprot.writeI32(this.launch_time_secs);
-      oprot.writeFieldEnd();
-    }
-    if (this.owner != null) {
-      if (is_set_owner()) {
-        oprot.writeFieldBegin(OWNER_FIELD_DESC);
-        oprot.writeString(this.owner);
-        oprot.writeFieldEnd();
-      }
-    }
-    if (this.topology_action_options != null) {
-      if (is_set_topology_action_options()) {
-        oprot.writeFieldBegin(TOPOLOGY_ACTION_OPTIONS_FIELD_DESC);
-        this.topology_action_options.write(oprot);
-        oprot.writeFieldEnd();
-      }
-    }
-    if (this.prev_status != null) {
-      if (is_set_prev_status()) {
-        oprot.writeFieldBegin(PREV_STATUS_FIELD_DESC);
-        oprot.writeI32(this.prev_status.getValue());
-        oprot.writeFieldEnd();
-      }
-    }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -1054,6 +918,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString());
     }
 
+    // check for sub-struct validity
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -1067,12 +932,280 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   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_bit_vector = new BitSet(1);
+      __isset_bitfield = 0;
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
     }
   }
 
+  private static class StormBaseStandardSchemeFactory implements SchemeFactory {
+    public StormBaseStandardScheme getScheme() {
+      return new StormBaseStandardScheme();
+    }
+  }
+
+  private static class StormBaseStandardScheme extends StandardScheme<StormBase> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, StormBase struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.status = backtype.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+              struct.set_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NUM_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_workers = iprot.readI32();
+              struct.set_num_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // COMPONENT_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map416 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map416.size);
+                String _key417;
+                int _val418;
+                for (int _i419 = 0; _i419 < _map416.size; ++_i419)
+                {
+                  _key417 = iprot.readString();
+                  _val418 = iprot.readI32();
+                  struct.component_executors.put(_key417, _val418);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_component_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // LAUNCH_TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.launch_time_secs = iprot.readI32();
+              struct.set_launch_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // OWNER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.owner = iprot.readString();
+              struct.set_owner_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // TOPOLOGY_ACTION_OPTIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.topology_action_options = new TopologyActionOptions();
+              struct.topology_action_options.read(iprot);
+              struct.set_topology_action_options_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // PREV_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.prev_status = backtype.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+              struct.set_prev_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, StormBase struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.status != null) {
+        oprot.writeFieldBegin(STATUS_FIELD_DESC);
+        oprot.writeI32(struct.status.getValue());
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+      oprot.writeI32(struct.num_workers);
+      oprot.writeFieldEnd();
+      if (struct.component_executors != null) {
+        if (struct.is_set_component_executors()) {
+          oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.component_executors.size()));
+            for (Map.Entry<String, Integer> _iter420 : struct.component_executors.entrySet())
+            {
+              oprot.writeString(_iter420.getKey());
+              oprot.writeI32(_iter420.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_launch_time_secs()) {
+        oprot.writeFieldBegin(LAUNCH_TIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.launch_time_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.owner != null) {
+        if (struct.is_set_owner()) {
+          oprot.writeFieldBegin(OWNER_FIELD_DESC);
+          oprot.writeString(struct.owner);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_action_options != null) {
+        if (struct.is_set_topology_action_options()) {
+          oprot.writeFieldBegin(TOPOLOGY_ACTION_OPTIONS_FIELD_DESC);
+          struct.topology_action_options.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.prev_status != null) {
+        if (struct.is_set_prev_status()) {
+          oprot.writeFieldBegin(PREV_STATUS_FIELD_DESC);
+          oprot.writeI32(struct.prev_status.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class StormBaseTupleSchemeFactory implements SchemeFactory {
+    public StormBaseTupleScheme getScheme() {
+      return new StormBaseTupleScheme();
+    }
+  }
+
+  private static class StormBaseTupleScheme extends TupleScheme<StormBase> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, StormBase struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.name);
+      oprot.writeI32(struct.status.getValue());
+      oprot.writeI32(struct.num_workers);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_component_executors()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_launch_time_secs()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_owner()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_topology_action_options()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_prev_status()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.is_set_component_executors()) {
+        {
+          oprot.writeI32(struct.component_executors.size());
+          for (Map.Entry<String, Integer> _iter421 : struct.component_executors.entrySet())
+          {
+            oprot.writeString(_iter421.getKey());
+            oprot.writeI32(_iter421.getValue());
+          }
+        }
+      }
+      if (struct.is_set_launch_time_secs()) {
+        oprot.writeI32(struct.launch_time_secs);
+      }
+      if (struct.is_set_owner()) {
+        oprot.writeString(struct.owner);
+      }
+      if (struct.is_set_topology_action_options()) {
+        struct.topology_action_options.write(oprot);
+      }
+      if (struct.is_set_prev_status()) {
+        oprot.writeI32(struct.prev_status.getValue());
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, StormBase struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.name = iprot.readString();
+      struct.set_name_isSet(true);
+      struct.status = backtype.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+      struct.set_status_isSet(true);
+      struct.num_workers = iprot.readI32();
+      struct.set_num_workers_isSet(true);
+      BitSet incoming = iprot.readBitSet(5);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map422 = 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*_map422.size);
+          String _key423;
+          int _val424;
+          for (int _i425 = 0; _i425 < _map422.size; ++_i425)
+          {
+            _key423 = iprot.readString();
+            _val424 = iprot.readI32();
+            struct.component_executors.put(_key423, _val424);
+          }
+        }
+        struct.set_component_executors_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.launch_time_secs = iprot.readI32();
+        struct.set_launch_time_secs_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.owner = iprot.readString();
+        struct.set_owner_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.topology_action_options = new TopologyActionOptions();
+        struct.topology_action_options.read(iprot);
+        struct.set_topology_action_options_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.prev_status = backtype.storm.generated.TopologyStatus.findByValue(iprot.readI32());
+        struct.set_prev_status_isSet(true);
+      }
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/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 523b1c0..924d12f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
@@ -16,13 +16,24 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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.commons.lang.builder.HashCodeBuilder;
+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;
@@ -35,10 +46,13 @@ 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;
 
-public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable {
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo");
 
   private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I64, (short)1);
@@ -49,13 +63,19 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   private static final org.apache.thrift.protocol.TField SCHEDULER_META_FIELD_DESC = new org.apache.thrift.protocol.TField("scheduler_meta", org.apache.thrift.protocol.TType.MAP, (short)6);
   private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I64, (short)7);
 
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SupervisorInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SupervisorInfoTupleSchemeFactory());
+  }
+
   private long time_secs; // required
   private String hostname; // required
-  private String assignment_id; // required
-  private List<Long> used_ports; // required
-  private List<Long> meta; // required
-  private Map<String,String> scheduler_meta; // required
-  private long uptime_secs; // required
+  private String assignment_id; // optional
+  private List<Long> used_ports; // optional
+  private List<Long> meta; // optional
+  private Map<String,String> scheduler_meta; // optional
+  private long uptime_secs; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -136,28 +156,28 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   // isset id assignments
   private static final int __TIME_SECS_ISSET_ID = 0;
   private static final int __UPTIME_SECS_ISSET_ID = 1;
-  private BitSet __isset_bit_vector = new BitSet(2);
-
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ASSIGNMENT_ID,_Fields.USED_PORTS,_Fields.META,_Fields.SCHEDULER_META,_Fields.UPTIME_SECS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-    tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.ASSIGNMENT_ID, new org.apache.thrift.meta_data.FieldMetaData("assignment_id", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.ASSIGNMENT_ID, new org.apache.thrift.meta_data.FieldMetaData("assignment_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.USED_PORTS, new org.apache.thrift.meta_data.FieldMetaData("used_ports", org.apache.thrift.TFieldRequirementType.OPTIONAL,
-        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+    tmpMap.put(_Fields.USED_PORTS, new org.apache.thrift.meta_data.FieldMetaData("used_ports", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
-    tmpMap.put(_Fields.META, new org.apache.thrift.meta_data.FieldMetaData("meta", org.apache.thrift.TFieldRequirementType.OPTIONAL,
-        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+    tmpMap.put(_Fields.META, new org.apache.thrift.meta_data.FieldMetaData("meta", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
-    tmpMap.put(_Fields.SCHEDULER_META, new org.apache.thrift.meta_data.FieldMetaData("scheduler_meta", org.apache.thrift.TFieldRequirementType.OPTIONAL,
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+    tmpMap.put(_Fields.SCHEDULER_META, new org.apache.thrift.meta_data.FieldMetaData("scheduler_meta", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorInfo.class, metaDataMap);
@@ -180,8 +200,7 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
    * Performs a deep copy on <i>other</i>.
    */
   public SupervisorInfo(SupervisorInfo other) {
-    __isset_bit_vector.clear();
-    __isset_bit_vector.or(other.__isset_bit_vector);
+    __isset_bitfield = other.__isset_bitfield;
     this.time_secs = other.time_secs;
     if (other.is_set_hostname()) {
       this.hostname = other.hostname;
@@ -190,32 +209,15 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       this.assignment_id = other.assignment_id;
     }
     if (other.is_set_used_ports()) {
-      List<Long> __this__used_ports = new ArrayList<Long>();
-      for (Long other_element : other.used_ports) {
-        __this__used_ports.add(other_element);
-      }
+      List<Long> __this__used_ports = new ArrayList<Long>(other.used_ports);
       this.used_ports = __this__used_ports;
     }
     if (other.is_set_meta()) {
-      List<Long> __this__meta = new ArrayList<Long>();
-      for (Long other_element : other.meta) {
-        __this__meta.add(other_element);
-      }
+      List<Long> __this__meta = new ArrayList<Long>(other.meta);
       this.meta = __this__meta;
     }
     if (other.is_set_scheduler_meta()) {
-      Map<String,String> __this__scheduler_meta = new HashMap<String,String>();
-      for (Map.Entry<String, String> other_element : other.scheduler_meta.entrySet()) {
-
-        String other_element_key = other_element.getKey();
-        String other_element_value = other_element.getValue();
-
-        String __this__scheduler_meta_copy_key = other_element_key;
-
-        String __this__scheduler_meta_copy_value = other_element_value;
-
-        __this__scheduler_meta.put(__this__scheduler_meta_copy_key, __this__scheduler_meta_copy_value);
-      }
+      Map<String,String> __this__scheduler_meta = new HashMap<String,String>(other.scheduler_meta);
       this.scheduler_meta = __this__scheduler_meta;
     }
     this.uptime_secs = other.uptime_secs;
@@ -248,16 +250,16 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   }
 
   public void unset_time_secs() {
-    __isset_bit_vector.clear(__TIME_SECS_ISSET_ID);
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
   }
 
   /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
   public boolean is_set_time_secs() {
-    return __isset_bit_vector.get(__TIME_SECS_ISSET_ID);
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
   }
 
   public void set_time_secs_isSet(boolean value) {
-    __isset_bit_vector.set(__TIME_SECS_ISSET_ID, value);
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
   }
 
   public String get_hostname() {
@@ -426,16 +428,16 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   }
 
   public void unset_uptime_secs() {
-    __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID);
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
   }
 
   /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
   public boolean is_set_uptime_secs() {
-    return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID);
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
   }
 
   public void set_uptime_secs_isSet(boolean value) {
-    __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value);
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
   }
 
   public void setFieldValue(_Fields field, Object value) {
@@ -632,120 +634,120 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
+    List<Object> list = new ArrayList<Object>();
 
     boolean present_time_secs = true;
-    builder.append(present_time_secs);
+    list.add(present_time_secs);
     if (present_time_secs)
-      builder.append(time_secs);
+      list.add(time_secs);
 
     boolean present_hostname = true && (is_set_hostname());
-    builder.append(present_hostname);
+    list.add(present_hostname);
     if (present_hostname)
-      builder.append(hostname);
+      list.add(hostname);
 
     boolean present_assignment_id = true && (is_set_assignment_id());
-    builder.append(present_assignment_id);
+    list.add(present_assignment_id);
     if (present_assignment_id)
-      builder.append(assignment_id);
+      list.add(assignment_id);
 
     boolean present_used_ports = true && (is_set_used_ports());
-    builder.append(present_used_ports);
+    list.add(present_used_ports);
     if (present_used_ports)
-      builder.append(used_ports);
+      list.add(used_ports);
 
     boolean present_meta = true && (is_set_meta());
-    builder.append(present_meta);
+    list.add(present_meta);
     if (present_meta)
-      builder.append(meta);
+      list.add(meta);
 
     boolean present_scheduler_meta = true && (is_set_scheduler_meta());
-    builder.append(present_scheduler_meta);
+    list.add(present_scheduler_meta);
     if (present_scheduler_meta)
-      builder.append(scheduler_meta);
+      list.add(scheduler_meta);
 
     boolean present_uptime_secs = true && (is_set_uptime_secs());
-    builder.append(present_uptime_secs);
+    list.add(present_uptime_secs);
     if (present_uptime_secs)
-      builder.append(uptime_secs);
+      list.add(uptime_secs);
 
-    return builder.toHashCode();
+    return list.hashCode();
   }
 
+  @Override
   public int compareTo(SupervisorInfo other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
 
     int lastComparison = 0;
-    SupervisorInfo typedOther = (SupervisorInfo)other;
 
-    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(typedOther.is_set_time_secs());
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_time_secs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, typedOther.time_secs);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_hostname()).compareTo(typedOther.is_set_hostname());
+    lastComparison = Boolean.valueOf(is_set_hostname()).compareTo(other.is_set_hostname());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_hostname()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, typedOther.hostname);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, other.hostname);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_assignment_id()).compareTo(typedOther.is_set_assignment_id());
+    lastComparison = Boolean.valueOf(is_set_assignment_id()).compareTo(other.is_set_assignment_id());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_assignment_id()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assignment_id, typedOther.assignment_id);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assignment_id, other.assignment_id);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_used_ports()).compareTo(typedOther.is_set_used_ports());
+    lastComparison = Boolean.valueOf(is_set_used_ports()).compareTo(other.is_set_used_ports());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_used_ports()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_ports, typedOther.used_ports);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_ports, other.used_ports);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_meta()).compareTo(typedOther.is_set_meta());
+    lastComparison = Boolean.valueOf(is_set_meta()).compareTo(other.is_set_meta());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_meta()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.meta, typedOther.meta);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.meta, other.meta);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_scheduler_meta()).compareTo(typedOther.is_set_scheduler_meta());
+    lastComparison = Boolean.valueOf(is_set_scheduler_meta()).compareTo(other.is_set_scheduler_meta());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_scheduler_meta()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scheduler_meta, typedOther.scheduler_meta);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scheduler_meta, other.scheduler_meta);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs());
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_uptime_secs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -758,176 +760,11 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) {
-        break;
-      }
-      switch (field.id) {
-        case 1: // TIME_SECS
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.time_secs = iprot.readI64();
-            set_time_secs_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // HOSTNAME
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.hostname = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 3: // ASSIGNMENT_ID
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.assignment_id = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 4: // USED_PORTS
-          if (field.type == org.apache.thrift.protocol.TType.LIST) {
-            {
-              org.apache.thrift.protocol.TList _list168 = iprot.readListBegin();
-              this.used_ports = new ArrayList<Long>(_list168.size);
-              for (int _i169 = 0; _i169 < _list168.size; ++_i169)
-              {
-                long _elem170; // required
-                _elem170 = iprot.readI64();
-                this.used_ports.add(_elem170);
-              }
-              iprot.readListEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 5: // META
-          if (field.type == org.apache.thrift.protocol.TType.LIST) {
-            {
-              org.apache.thrift.protocol.TList _list171 = iprot.readListBegin();
-              this.meta = new ArrayList<Long>(_list171.size);
-              for (int _i172 = 0; _i172 < _list171.size; ++_i172)
-              {
-                long _elem173; // required
-                _elem173 = iprot.readI64();
-                this.meta.add(_elem173);
-              }
-              iprot.readListEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 6: // SCHEDULER_META
-          if (field.type == org.apache.thrift.protocol.TType.MAP) {
-            {
-              org.apache.thrift.protocol.TMap _map174 = iprot.readMapBegin();
-              this.scheduler_meta = new HashMap<String,String>(2*_map174.size);
-              for (int _i175 = 0; _i175 < _map174.size; ++_i175)
-              {
-                String _key176; // required
-                String _val177; // required
-                _key176 = iprot.readString();
-                _val177 = iprot.readString();
-                this.scheduler_meta.put(_key176, _val177);
-              }
-              iprot.readMapEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 7: // UPTIME_SECS
-          if (field.type == org.apache.thrift.protocol.TType.I64) {
-            this.uptime_secs = iprot.readI64();
-            set_uptime_secs_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
-    oprot.writeI64(this.time_secs);
-    oprot.writeFieldEnd();
-    if (this.hostname != null) {
-      oprot.writeFieldBegin(HOSTNAME_FIELD_DESC);
-      oprot.writeString(this.hostname);
-      oprot.writeFieldEnd();
-    }
-    if (this.assignment_id != null) {
-      if (is_set_assignment_id()) {
-        oprot.writeFieldBegin(ASSIGNMENT_ID_FIELD_DESC);
-        oprot.writeString(this.assignment_id);
-        oprot.writeFieldEnd();
-      }
-    }
-    if (this.used_ports != null) {
-      if (is_set_used_ports()) {
-        oprot.writeFieldBegin(USED_PORTS_FIELD_DESC);
-        {
-          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, this.used_ports.size()));
-          for (long _iter178 : this.used_ports)
-          {
-            oprot.writeI64(_iter178);
-          }
-          oprot.writeListEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-    }
-    if (this.meta != null) {
-      if (is_set_meta()) {
-        oprot.writeFieldBegin(META_FIELD_DESC);
-        {
-          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, this.meta.size()));
-          for (long _iter179 : this.meta)
-          {
-            oprot.writeI64(_iter179);
-          }
-          oprot.writeListEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-    }
-    if (this.scheduler_meta != null) {
-      if (is_set_scheduler_meta()) {
-        oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC);
-        {
-          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, this.scheduler_meta.size()));
-          for (Map.Entry<String, String> _iter180 : this.scheduler_meta.entrySet())
-          {
-            oprot.writeString(_iter180.getKey());
-            oprot.writeString(_iter180.getValue());
-          }
-          oprot.writeMapEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-    }
-    if (is_set_uptime_secs()) {
-      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
-      oprot.writeI64(this.uptime_secs);
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -1006,6 +843,7 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString());
     }
 
+    // check for sub-struct validity
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -1019,12 +857,326 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   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_bit_vector = new BitSet(1);
+      __isset_bitfield = 0;
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
     }
   }
 
+  private static class SupervisorInfoStandardSchemeFactory implements SchemeFactory {
+    public SupervisorInfoStandardScheme getScheme() {
+      return new SupervisorInfoStandardScheme();
+    }
+  }
+
+  private static class SupervisorInfoStandardScheme extends StandardScheme<SupervisorInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.time_secs = iprot.readI64();
+              struct.set_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // HOSTNAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.hostname = iprot.readString();
+              struct.set_hostname_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ASSIGNMENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.assignment_id = iprot.readString();
+              struct.set_assignment_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // USED_PORTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list336 = iprot.readListBegin();
+                struct.used_ports = new ArrayList<Long>(_list336.size);
+                long _elem337;
+                for (int _i338 = 0; _i338 < _list336.size; ++_i338)
+                {
+                  _elem337 = iprot.readI64();
+                  struct.used_ports.add(_elem337);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_used_ports_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // META
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list339 = iprot.readListBegin();
+                struct.meta = new ArrayList<Long>(_list339.size);
+                long _elem340;
+                for (int _i341 = 0; _i341 < _list339.size; ++_i341)
+                {
+                  _elem340 = iprot.readI64();
+                  struct.meta.add(_elem340);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_meta_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // SCHEDULER_META
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map342 = iprot.readMapBegin();
+                struct.scheduler_meta = new HashMap<String,String>(2*_map342.size);
+                String _key343;
+                String _val344;
+                for (int _i345 = 0; _i345 < _map342.size; ++_i345)
+                {
+                  _key343 = iprot.readString();
+                  _val344 = iprot.readString();
+                  struct.scheduler_meta.put(_key343, _val344);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_scheduler_meta_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.uptime_secs = iprot.readI64();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+      oprot.writeI64(struct.time_secs);
+      oprot.writeFieldEnd();
+      if (struct.hostname != null) {
+        oprot.writeFieldBegin(HOSTNAME_FIELD_DESC);
+        oprot.writeString(struct.hostname);
+        oprot.writeFieldEnd();
+      }
+      if (struct.assignment_id != null) {
+        if (struct.is_set_assignment_id()) {
+          oprot.writeFieldBegin(ASSIGNMENT_ID_FIELD_DESC);
+          oprot.writeString(struct.assignment_id);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.used_ports != null) {
+        if (struct.is_set_used_ports()) {
+          oprot.writeFieldBegin(USED_PORTS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.used_ports.size()));
+            for (long _iter346 : struct.used_ports)
+            {
+              oprot.writeI64(_iter346);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.meta != null) {
+        if (struct.is_set_meta()) {
+          oprot.writeFieldBegin(META_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size()));
+            for (long _iter347 : struct.meta)
+            {
+              oprot.writeI64(_iter347);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.scheduler_meta != null) {
+        if (struct.is_set_scheduler_meta()) {
+          oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.scheduler_meta.size()));
+            for (Map.Entry<String, String> _iter348 : struct.scheduler_meta.entrySet())
+            {
+              oprot.writeString(_iter348.getKey());
+              oprot.writeString(_iter348.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+        oprot.writeI64(struct.uptime_secs);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SupervisorInfoTupleSchemeFactory implements SchemeFactory {
+    public SupervisorInfoTupleScheme getScheme() {
+      return new SupervisorInfoTupleScheme();
+    }
+  }
+
+  private static class SupervisorInfoTupleScheme extends TupleScheme<SupervisorInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.time_secs);
+      oprot.writeString(struct.hostname);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_assignment_id()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_used_ports()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_meta()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_scheduler_meta()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_uptime_secs()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.is_set_assignment_id()) {
+        oprot.writeString(struct.assignment_id);
+      }
+      if (struct.is_set_used_ports()) {
+        {
+          oprot.writeI32(struct.used_ports.size());
+          for (long _iter349 : struct.used_ports)
+          {
+            oprot.writeI64(_iter349);
+          }
+        }
+      }
+      if (struct.is_set_meta()) {
+        {
+          oprot.writeI32(struct.meta.size());
+          for (long _iter350 : struct.meta)
+          {
+            oprot.writeI64(_iter350);
+          }
+        }
+      }
+      if (struct.is_set_scheduler_meta()) {
+        {
+          oprot.writeI32(struct.scheduler_meta.size());
+          for (Map.Entry<String, String> _iter351 : struct.scheduler_meta.entrySet())
+          {
+            oprot.writeString(_iter351.getKey());
+            oprot.writeString(_iter351.getValue());
+          }
+        }
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeI64(struct.uptime_secs);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.time_secs = iprot.readI64();
+      struct.set_time_secs_isSet(true);
+      struct.hostname = iprot.readString();
+      struct.set_hostname_isSet(true);
+      BitSet incoming = iprot.readBitSet(5);
+      if (incoming.get(0)) {
+        struct.assignment_id = iprot.readString();
+        struct.set_assignment_id_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list352 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.used_ports = new ArrayList<Long>(_list352.size);
+          long _elem353;
+          for (int _i354 = 0; _i354 < _list352.size; ++_i354)
+          {
+            _elem353 = iprot.readI64();
+            struct.used_ports.add(_elem353);
+          }
+        }
+        struct.set_used_ports_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list355 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.meta = new ArrayList<Long>(_list355.size);
+          long _elem356;
+          for (int _i357 = 0; _i357 < _list355.size; ++_i357)
+          {
+            _elem356 = iprot.readI64();
+            struct.meta.add(_elem356);
+          }
+        }
+        struct.set_meta_isSet(true);
+      }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TMap _map358 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.scheduler_meta = new HashMap<String,String>(2*_map358.size);
+          String _key359;
+          String _val360;
+          for (int _i361 = 0; _i361 < _map358.size; ++_i361)
+          {
+            _key359 = iprot.readString();
+            _val360 = iprot.readString();
+            struct.scheduler_meta.put(_key359, _val360);
+          }
+        }
+        struct.set_scheduler_meta_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.uptime_secs = iprot.readI64();
+        struct.set_uptime_secs_isSet(true);
+      }
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java b/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
index a3c65cf..c6fea05 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
@@ -16,13 +16,24 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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.commons.lang.builder.HashCodeBuilder;
+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;
@@ -35,9 +46,11 @@ import java.util.Collections;
 import java.util.BitSet;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import javax.annotation.Generated;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 public class TopologyActionOptions extends org.apache.thrift.TUnion<TopologyActionOptions, TopologyActionOptions._Fields> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyActionOptions");
   private static final org.apache.thrift.protocol.TField KILL_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("kill_options", org.apache.thrift.protocol.TType.STRUCT, (short)1);
@@ -107,9 +120,9 @@ public class TopologyActionOptions extends org.apache.thrift.TUnion<TopologyActi
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.KILL_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("kill_options", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.KILL_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("kill_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KillOptions.class)));
-    tmpMap.put(_Fields.REBALANCE_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("rebalance_options", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.REBALANCE_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("rebalance_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RebalanceOptions.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyActionOptions.class, metaDataMap);
@@ -162,7 +175,7 @@ public class TopologyActionOptions extends org.apache.thrift.TUnion<TopologyActi
   }
 
   @Override
-  protected Object readValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
     _Fields setField = _Fields.findByThriftId(field.id);
     if (setField != null) {
       switch (setField) {
@@ -196,7 +209,46 @@ public class TopologyActionOptions extends org.apache.thrift.TUnion<TopologyActi
   }
 
   @Override
-  protected void writeValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case KILL_OPTIONS:
+        KillOptions kill_options = (KillOptions)value_;
+        kill_options.write(oprot);
+        return;
+      case REBALANCE_OPTIONS:
+        RebalanceOptions rebalance_options = (RebalanceOptions)value_;
+        rebalance_options.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case KILL_OPTIONS:
+          KillOptions kill_options;
+          kill_options = new KillOptions();
+          kill_options.read(iprot);
+          return kill_options;
+        case REBALANCE_OPTIONS:
+          RebalanceOptions rebalance_options;
+          rebalance_options = new RebalanceOptions();
+          rebalance_options.read(iprot);
+          return rebalance_options;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
     switch (setField_) {
       case KILL_OPTIONS:
         KillOptions kill_options = (KillOptions)value_;
@@ -300,19 +352,19 @@ public class TopologyActionOptions extends org.apache.thrift.TUnion<TopologyActi
 
   @Override
   public int hashCode() {
-    HashCodeBuilder hcb = new HashCodeBuilder();
-    hcb.append(this.getClass().getName());
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
     org.apache.thrift.TFieldIdEnum setField = getSetField();
     if (setField != null) {
-      hcb.append(setField.getThriftFieldId());
+      list.add(setField.getThriftFieldId());
       Object value = getFieldValue();
       if (value instanceof org.apache.thrift.TEnum) {
-        hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
       } else {
-        hcb.append(value);
+        list.add(value);
       }
     }
-    return hcb.toHashCode();
+    return list.hashCode();
   }
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
     try {

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java b/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java
index c6bdd36..8183ab2 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java
@@ -16,9 +16,10 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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;
 


[15/16] storm git commit: Merge branch 'STORM-634' of https://github.com/Parth-Brahmbhatt/incubator-storm into STORM-634

Posted by bo...@apache.org.
Merge branch 'STORM-634' of https://github.com/Parth-Brahmbhatt/incubator-storm into STORM-634

STORM-634: Storm serialization changed to thrift to support rolling upgrade.


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

Branch: refs/heads/master
Commit: 66274f342c8d35e4e952f2d22a6250a1a6760287
Parents: 1ea378d 93b1b65
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Mar 18 14:20:15 2015 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Mar 18 14:20:15 2015 -0500

----------------------------------------------------------------------
 conf/defaults.yaml                              |    2 +-
 storm-core/src/clj/backtype/storm/cluster.clj   |   58 +-
 storm-core/src/clj/backtype/storm/config.clj    |    7 +-
 storm-core/src/clj/backtype/storm/converter.clj |  201 +++
 .../src/clj/backtype/storm/daemon/common.clj    |   10 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |    4 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  106 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |    3 +-
 storm-core/src/clj/backtype/storm/stats.clj     |   78 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |   17 +-
 .../storm/coordination/BatchBoltExecutor.java   |    4 +-
 .../backtype/storm/generated/Assignment.java    |  983 ++++++++++++++
 .../storm/generated/ClusterWorkerHeartbeat.java |  673 ++++++++++
 .../backtype/storm/generated/ExecutorStats.java |  105 +-
 .../jvm/backtype/storm/generated/NodeInfo.java  |  556 ++++++++
 .../jvm/backtype/storm/generated/StormBase.java | 1211 ++++++++++++++++++
 .../storm/generated/SupervisorInfo.java         | 1182 +++++++++++++++++
 .../storm/generated/TopologyActionOptions.java  |  387 ++++++
 .../storm/generated/TopologyStatus.java         |   68 +
 .../DefaultSerializationDelegate.java           |   11 +-
 .../GzipBridgeSerializationDelegate.java        |    7 +-
 .../GzipSerializationDelegate.java              |   10 +-
 .../serialization/SerializationDelegate.java    |    2 +-
 .../ThriftSerializationDelegate.java            |   52 +
 .../storm/topology/TopologyBuilder.java         |    6 +-
 .../jvm/backtype/storm/utils/LocalState.java    |    4 +-
 .../src/jvm/backtype/storm/utils/Utils.java     |   82 +-
 storm-core/src/jvm/storm/trident/Stream.java    |    2 +-
 .../src/jvm/storm/trident/TridentTopology.java  |    6 +-
 storm-core/src/py/storm/ttypes.py               |  827 +++++++++++-
 storm-core/src/storm.thrift                     |   51 +
 .../test/clj/backtype/storm/cluster_test.clj    |   25 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |    6 +-
 .../GzipBridgeSerializationDelegateTest.java    |    6 +-
 .../ThriftBridgeSerializationDelegateTest.java  |   60 +
 35 files changed, 6606 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/storm/blob/66274f34/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/66274f34/storm-core/src/clj/backtype/storm/thrift.clj
----------------------------------------------------------------------


[13/16] storm git commit: Deprecating DefaultSerializer and DefaultSerializerBridge in favor of ThriftSerializer.

Posted by bo...@apache.org.
Deprecating DefaultSerializer and DefaultSerializerBridge in favor of ThriftSerializer.


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

Branch: refs/heads/master
Commit: 47bcf3ebe7067e86846057cbca2466d7349b0d98
Parents: 5a66bb6
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Mar 11 14:06:49 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Mar 11 14:06:49 2015 -0700

----------------------------------------------------------------------
 conf/defaults.yaml                              |  2 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |  2 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  4 +-
 .../storm/coordination/BatchBoltExecutor.java   |  2 +-
 .../DefaultSerializationDelegate.java           |  1 +
 .../GzipBridgeSerializationDelegate.java        |  1 +
 .../ThriftSerializationDelegateBridge.java      | 51 --------------------
 .../storm/topology/TopologyBuilder.java         |  2 +-
 .../src/jvm/backtype/storm/utils/Utils.java     | 23 +++++++--
 .../ThriftBridgeSerializationDelegateTest.java  | 21 +-------
 10 files changed, 29 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 0255a55..dfdb54b 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -47,7 +47,7 @@ storm.auth.simple-white-list.users: []
 storm.auth.simple-acl.users: []
 storm.auth.simple-acl.users.commands: []
 storm.auth.simple-acl.admins: []
-storm.meta.serialization.delegate: "backtype.storm.serialization.ThriftSerializationDelegateBridge"
+storm.meta.serialization.delegate: "backtype.storm.serialization.ThriftSerializationDelegate"
 
 ### nimbus.* configs are for the master
 nimbus.host: "localhost"

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index 0e083ea..7f688ed 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -96,7 +96,7 @@
         (let [grouping (thrift/instantiate-java-object (.get_custom_object thrift-grouping))]
           (mk-custom-grouper grouping context component-id stream-id target-tasks))
       :custom-serialized
-        (let [grouping (Utils/deserialize (.get_custom_serialized thrift-grouping) Serializable)]
+        (let [grouping (Utils/javaDeserialize (.get_custom_serialized thrift-grouping) Serializable)]
           (mk-custom-grouper grouping context component-id stream-id target-tasks))
       :direct
         :direct

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/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 fdc54cc..9112365 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -107,7 +107,7 @@
 (defn- read-storm-conf [conf storm-id]
   (let [stormroot (master-stormdist-root conf storm-id)]
     (merge conf
-           (Utils/deserialize
+           (Utils/javaDeserialize
             (FileUtils/readFileToByteArray
              (File. (master-stormconf-path stormroot))
              ) java.util.Map))))
@@ -322,7 +322,7 @@
    (FileUtils/cleanDirectory (File. stormroot))
    (setup-jar conf tmp-jar-location stormroot)
    (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
-   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/serialize storm-conf))
+   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/javaSerialize storm-conf))
    ))
 
 (defn- read-storm-topology [conf storm-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
index 3d3208b..89ef028 100644
--- a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
@@ -41,7 +41,7 @@ public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCa
     BatchOutputCollectorImpl _collector;
     
     public BatchBoltExecutor(IBatchBolt bolt) {
-        _boltSer = Utils.serialize(bolt);
+        _boltSer = Utils.javaSerialize(bolt);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
index 913baa6..6d986af 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/DefaultSerializationDelegate.java
@@ -20,6 +20,7 @@ package backtype.storm.serialization;
 import java.io.*;
 import java.util.Map;
 
+@Deprecated
 public class DefaultSerializationDelegate implements SerializationDelegate {
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
index 1306ea9..c8377c3 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/GzipBridgeSerializationDelegate.java
@@ -25,6 +25,7 @@ import java.util.zip.GZIPInputStream;
  * {@link backtype.storm.serialization.DefaultSerializationDelegate} to deserialize. Any logic needing to be enabled
  * via {@link #prepare(java.util.Map)} is passed through to both delegates.
  */
+@Deprecated
 public class GzipBridgeSerializationDelegate implements SerializationDelegate {
 
     private DefaultSerializationDelegate defaultDelegate = new DefaultSerializationDelegate();

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java b/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java
deleted file mode 100644
index 8b8c95d..0000000
--- a/storm-core/src/jvm/backtype/storm/serialization/ThriftSerializationDelegateBridge.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package backtype.storm.serialization;
-
-import org.apache.thrift.TBase;
-
-import java.util.Map;
-
-public class ThriftSerializationDelegateBridge implements SerializationDelegate {
-    private SerializationDelegate thriftSerializationDelegate = new ThriftSerializationDelegate();
-    private SerializationDelegate defaultSerializationDelegate = new DefaultSerializationDelegate();
-
-    @Override
-    public void prepare(Map stormConf) {
-        this.thriftSerializationDelegate.prepare(stormConf);
-        this.defaultSerializationDelegate.prepare(stormConf);
-    }
-
-    @Override
-    public byte[] serialize(Object object) {
-        if(object instanceof TBase) {
-            return thriftSerializationDelegate.serialize(object);
-        } else {
-            return defaultSerializationDelegate.serialize(object);
-        }
-    }
-
-    @Override
-    public <T> T deserialize(byte[] bytes, Class<T> clazz) {
-        if(TBase.class.isAssignableFrom(clazz)) {
-            return thriftSerializationDelegate.deserialize(bytes, clazz);
-        } else {
-            return defaultSerializationDelegate.deserialize(bytes, clazz);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java b/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
index 9d8f271..806549a 100644
--- a/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
+++ b/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
@@ -348,7 +348,7 @@ public class TopologyBuilder {
 
         @Override
         public BoltDeclarer customGrouping(String componentId, String streamId, CustomStreamGrouping grouping) {
-            return grouping(componentId, streamId, Grouping.custom_serialized(Utils.serialize(grouping)));
+            return grouping(componentId, streamId, Grouping.custom_serialized(Utils.javaSerialize(grouping)));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index ba1a2ab..4123f73 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -52,7 +52,6 @@ public class Utils {
     public static final String DEFAULT_STREAM_ID = "default";
 
     private static SerializationDelegate serializationDelegate;
-    private static final DefaultSerializationDelegate javaSerializationDelegate = new DefaultSerializationDelegate();
 
     static {
         Map conf = readStormConfig();
@@ -77,11 +76,29 @@ public class Utils {
     }
 
     public static byte[] javaSerialize(Object obj) {
-        return javaSerializationDelegate.serialize(obj);
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            ObjectOutputStream oos = new ObjectOutputStream(bos);
+            oos.writeObject(obj);
+            oos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
     }
 
     public static <T> T javaDeserialize(byte[] serialized, Class<T> clazz) {
-        return javaSerializationDelegate.deserialize(serialized, clazz);
+        try {
+            ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+            ObjectInputStream ois = new ObjectInputStream(bis);
+            Object ret = ois.readObject();
+            ois.close();
+            return (T)ret;
+        } catch(IOException ioe) {
+            throw new RuntimeException(ioe);
+        } catch(ClassNotFoundException e) {
+            throw new RuntimeException(e);
+        }
     }
 
     public static <T> String join(Iterable<T> coll, String sep) {

http://git-wip-us.apache.org/repos/asf/storm/blob/47bcf3eb/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java b/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java
index ef17017..b408a80 100644
--- a/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java
+++ b/storm-core/test/jvm/backtype/storm/serialization/ThriftBridgeSerializationDelegateTest.java
@@ -32,30 +32,11 @@ public class ThriftBridgeSerializationDelegateTest {
 
     @Before
     public void setUp() throws Exception {
-        testDelegate = new ThriftSerializationDelegateBridge();
+        testDelegate = new ThriftSerializationDelegate();
         testDelegate.prepare(null);
     }
 
     @Test
-    public void testNonThriftInstance() throws Exception {
-        TestPojo pojo = new TestPojo();
-        pojo.name = "foo";
-        pojo.age = 100;
-
-        byte[] serialized = new DefaultSerializationDelegate().serialize(pojo);
-
-        TestPojo pojo2 = (TestPojo)testDelegate.deserialize(serialized, TestPojo.class);
-
-        assertEquals(pojo2.name, pojo.name);
-        assertEquals(pojo2.age, pojo.age);
-
-        serialized = testDelegate.serialize(pojo);
-        pojo2 = (TestPojo) new DefaultSerializationDelegate().deserialize(serialized, Serializable.class);
-        assertEquals(pojo2.name, pojo.name);
-        assertEquals(pojo2.age, pojo.age);
-    }
-
-    @Test
     public void testThriftInstance() throws Exception {
         ErrorInfo errorInfo = new ErrorInfo();
         errorInfo.set_error("error");


[06/16] storm git commit: Using java serialization for all places where the contract with customer is to use java serialization.

Posted by bo...@apache.org.
Using java serialization for all places where the contract with customer is to use java serialization.


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

Branch: refs/heads/master
Commit: 431976607857d5d29913ba055082a546a22be16e
Parents: 0571e22
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Tue Feb 17 10:57:25 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Tue Feb 17 10:57:25 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/config.clj             |  2 +-
 storm-core/src/clj/backtype/storm/thrift.clj             | 10 +++++-----
 .../src/jvm/backtype/storm/topology/TopologyBuilder.java |  4 ++--
 storm-core/src/jvm/backtype/storm/utils/LocalState.java  |  4 ++--
 storm-core/src/jvm/backtype/storm/utils/Utils.java       | 11 ++++++++++-
 storm-core/src/jvm/storm/trident/Stream.java             |  2 +-
 storm-core/src/jvm/storm/trident/TridentTopology.java    |  4 ++--
 7 files changed, 23 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/43197660/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index d09b31b..00396c7 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -212,7 +212,7 @@
   (let [stormroot (supervisor-stormdist-root conf storm-id)
         conf-path (supervisor-stormconf-path stormroot)
         topology-path (supervisor-stormcode-path stormroot)]
-    (merge conf (Utils/deserialize (FileUtils/readFileToByteArray (File. conf-path)) java.util.Map))
+    (merge conf (Utils/javaDeserialize (FileUtils/readFileToByteArray (File. conf-path)) java.util.Map))
     ))
 
 (defn read-supervisor-topology

http://git-wip-us.apache.org/repos/asf/storm/blob/43197660/storm-core/src/clj/backtype/storm/thrift.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/thrift.clj b/storm-core/src/clj/backtype/storm/thrift.clj
index 5bc1150..0ff6dd0 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -122,7 +122,7 @@
 
 (defnk mk-spout-spec*
   [spout outputs :p nil :conf nil]
-  (SpoutSpec. (ComponentObject/serialized_java (Utils/serialize spout))
+  (SpoutSpec. (ComponentObject/serialized_java (Utils/javaSerialize spout))
               (mk-plain-component-common {} outputs p :conf conf)))
 
 (defn mk-shuffle-grouping
@@ -157,11 +157,11 @@
   [^ComponentObject obj]
   (when (not= (.getSetField obj) ComponentObject$_Fields/SERIALIZED_JAVA)
     (throw (RuntimeException. "Cannot deserialize non-java-serialized object")))
-  (Utils/deserialize (.get_serialized_java obj) Serializable))
+  (Utils/javaDeserialize (.get_serialized_java obj) Serializable))
 
 (defn serialize-component-object
   [obj]
-  (ComponentObject/serialized_java (Utils/serialize obj)))
+  (ComponentObject/serialized_java (Utils/javaSerialize obj)))
 
 (defn- mk-grouping
   [grouping-spec]
@@ -172,7 +172,7 @@
         grouping-spec
 
         (instance? CustomStreamGrouping grouping-spec)
-        (Grouping/custom_serialized (Utils/serialize grouping-spec))
+        (Grouping/custom_serialized (Utils/javaSerialize grouping-spec))
 
         (instance? JavaObject grouping-spec)
         (Grouping/custom_object grouping-spec)
@@ -212,7 +212,7 @@
 (defnk mk-bolt-spec*
   [inputs bolt outputs :p nil :conf nil]
   (let [common (mk-plain-component-common (mk-inputs inputs) outputs p :conf conf)]
-    (Bolt. (ComponentObject/serialized_java (Utils/serialize bolt))
+    (Bolt. (ComponentObject/serialized_java (Utils/javaSerialize bolt))
            common)))
 
 (defnk mk-spout-spec

http://git-wip-us.apache.org/repos/asf/storm/blob/43197660/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java b/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
index 0a47626..9d8f271 100644
--- a/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
+++ b/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
@@ -104,12 +104,12 @@ public class TopologyBuilder {
         for(String boltId: _bolts.keySet()) {
             IRichBolt bolt = _bolts.get(boltId);
             ComponentCommon common = getComponentCommon(boltId, bolt);
-            boltSpecs.put(boltId, new Bolt(ComponentObject.serialized_java(Utils.serialize(bolt)), common));
+            boltSpecs.put(boltId, new Bolt(ComponentObject.serialized_java(Utils.javaSerialize(bolt)), common));
         }
         for(String spoutId: _spouts.keySet()) {
             IRichSpout spout = _spouts.get(spoutId);
             ComponentCommon common = getComponentCommon(spoutId, spout);
-            spoutSpecs.put(spoutId, new SpoutSpec(ComponentObject.serialized_java(Utils.serialize(spout)), common));
+            spoutSpecs.put(spoutId, new SpoutSpec(ComponentObject.serialized_java(Utils.javaSerialize(spout)), common));
             
         }
         return new StormTopology(spoutSpecs,

http://git-wip-us.apache.org/repos/asf/storm/blob/43197660/storm-core/src/jvm/backtype/storm/utils/LocalState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/LocalState.java b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
index 561988c..65f2152 100644
--- a/storm-core/src/jvm/backtype/storm/utils/LocalState.java
+++ b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
@@ -64,7 +64,7 @@ public class LocalState {
             if (serialized.length == 0) {
                 LOG.warn("LocalState file '{}' contained no data, resetting state", latestPath);
             } else {
-                result = Utils.deserialize(serialized, Map.class);
+                result = Utils.javaDeserialize(serialized, Map.class);
             }
         }
         return result;
@@ -99,7 +99,7 @@ public class LocalState {
     }
     
     private void persist(Map<Object, Object> val, boolean cleanup) throws IOException {
-        byte[] toWrite = Utils.serialize(val);
+        byte[] toWrite = Utils.javaSerialize(val);
         String newPath = _vs.createVersion();
         File file = new File(newPath);
         FileUtils.writeByteArrayToFile(file, toWrite);

http://git-wip-us.apache.org/repos/asf/storm/blob/43197660/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index fd56539..ba1a2ab 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -52,6 +52,7 @@ public class Utils {
     public static final String DEFAULT_STREAM_ID = "default";
 
     private static SerializationDelegate serializationDelegate;
+    private static final DefaultSerializationDelegate javaSerializationDelegate = new DefaultSerializationDelegate();
 
     static {
         Map conf = readStormConfig();
@@ -75,6 +76,14 @@ public class Utils {
         return serializationDelegate.deserialize(serialized, clazz);
     }
 
+    public static byte[] javaSerialize(Object obj) {
+        return javaSerializationDelegate.serialize(obj);
+    }
+
+    public static <T> T javaDeserialize(byte[] serialized, Class<T> clazz) {
+        return javaSerializationDelegate.deserialize(serialized, clazz);
+    }
+
     public static <T> String join(Iterable<T> coll, String sep) {
         Iterator<T> it = coll.iterator();
         String ret = "";
@@ -211,7 +220,7 @@ public class Utils {
 
     public static Object getSetComponentObject(ComponentObject obj) {
         if(obj.getSetField()==ComponentObject._Fields.SERIALIZED_JAVA) {
-            return Utils.deserialize(obj.get_serialized_java(), Serializable.class);
+            return Utils.javaDeserialize(obj.get_serialized_java(), Serializable.class);
         } else if(obj.getSetField()==ComponentObject._Fields.JAVA_OBJECT) {
             return obj.get_java_object();
         } else {

http://git-wip-us.apache.org/repos/asf/storm/blob/43197660/storm-core/src/jvm/storm/trident/Stream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/Stream.java b/storm-core/src/jvm/storm/trident/Stream.java
index c308745..f4c515e 100644
--- a/storm-core/src/jvm/storm/trident/Stream.java
+++ b/storm-core/src/jvm/storm/trident/Stream.java
@@ -94,7 +94,7 @@ public class Stream implements IAggregatableStream {
     }
     
     public Stream partition(CustomStreamGrouping partitioner) {
-        return partition(Grouping.custom_serialized(Utils.serialize(partitioner)));
+        return partition(Grouping.custom_serialized(Utils.javaSerialize(partitioner)));
     }
     
     public Stream shuffle() {

http://git-wip-us.apache.org/repos/asf/storm/blob/43197660/storm-core/src/jvm/storm/trident/TridentTopology.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/TridentTopology.java b/storm-core/src/jvm/storm/trident/TridentTopology.java
index 25506c8..3ed2386 100644
--- a/storm-core/src/jvm/storm/trident/TridentTopology.java
+++ b/storm-core/src/jvm/storm/trident/TridentTopology.java
@@ -666,7 +666,7 @@ public class TridentTopology {
     private static boolean isIdentityPartition(PartitionNode n) {
         Grouping g = n.thriftGrouping;
         if(g.is_set_custom_serialized()) {
-            CustomStreamGrouping csg = (CustomStreamGrouping) Utils.deserialize(g.get_custom_serialized(), Serializable.class);
+            CustomStreamGrouping csg = (CustomStreamGrouping) Utils.javaDeserialize(g.get_custom_serialized(), Serializable.class);
             return csg instanceof IdentityGrouping;
         }
         return false;
@@ -725,7 +725,7 @@ public class TridentTopology {
     
     private static PartitionNode makeIdentityPartition(Node basis) {
         return new PartitionNode(basis.streamId, basis.name, basis.allOutputFields,
-            Grouping.custom_serialized(Utils.serialize(new IdentityGrouping())));
+            Grouping.custom_serialized(Utils.javaSerialize(new IdentityGrouping())));
     }
     
     


[14/16] storm git commit: Fixing a failing unit test.

Posted by bo...@apache.org.
Fixing a failing unit test.


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

Branch: refs/heads/master
Commit: 93b1b656013a941b03c51936ea2c95cd7952fbc5
Parents: 47bcf3e
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Mar 18 10:34:56 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Mar 18 10:34:56 2015 -0700

----------------------------------------------------------------------
 .../src/jvm/backtype/storm/coordination/BatchBoltExecutor.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/93b1b656/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
index 89ef028..8653010 100644
--- a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
@@ -103,6 +103,6 @@ public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCa
     }
     
     private IBatchBolt newTransactionalBolt() {
-        return Utils.deserialize(_boltSer, IBatchBolt.class);
+        return Utils.javaDeserialize(_boltSer, IBatchBolt.class);
     }
 }


[07/16] storm git commit: Handling the case where executor stats can be null on initialization.

Posted by bo...@apache.org.
Handling the case where executor stats can be null on initialization.


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

Branch: refs/heads/master
Commit: f23b3c8719b94fa7af04ac2138479842c449deab
Parents: 4319766
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Mar 5 16:57:17 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Mar 5 16:57:17 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   | 12 +++++++-----
 storm-core/src/clj/backtype/storm/converter.clj |  9 +++++----
 2 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f23b3c87/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index 1d5026f..8866d04 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -336,10 +336,11 @@
 
       (get-worker-heartbeat
         [this storm-id node port]
-        (-> cluster-state
-            (get-data (workerbeat-path storm-id node port) false)
-          (maybe-deserialize ZKWorkerHeartbeat)
-          clojurify-zk-worker-hb))
+        (let [worker-hb (get-data cluster-state (workerbeat-path storm-id node port) false)]
+          (if worker-hb
+            (-> worker-hb
+              (maybe-deserialize ZKWorkerHeartbeat)
+              clojurify-zk-worker-hb))))
 
 
       (executor-beats
@@ -368,7 +369,8 @@
       (worker-heartbeat!
         [this storm-id node port info]
         (let [thrift-worker-hb (thriftify-zk-worker-hb info)]
-          (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize thrift-worker-hb) acls)))
+          (if thrift-worker-hb
+            (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize thrift-worker-hb) acls))))
 
       (remove-worker-heartbeat!
         [this storm-id node port]

http://git-wip-us.apache.org/repos/asf/storm/blob/f23b3c87/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 6a9f4a6..e011798 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -168,10 +168,11 @@
     {}))
 
 (defn thriftify-zk-worker-hb [worker-hb]
-  (doto (ZKWorkerHeartbeat.)
-    (.set_storm_id (:storm-id worker-hb))
-    (.set_executor_stats (thriftify-stats (:executor-stats worker-hb)))
-    (.set_time_secs (:time-secs worker-hb))))
+  (if (not-empty (filter second (:executor-stats worker-hb)))
+    (doto (ZKWorkerHeartbeat.)
+      (.set_storm_id (:storm-id worker-hb))
+      (.set_executor_stats (thriftify-stats (filter second (:executor-stats worker-hb))))
+      (.set_time_secs (:time-secs worker-hb)))))
 
 (defn clojurify-error [^ErrorInfo error]
   (if error


[16/16] storm git commit: added STORM-634 to Changelog

Posted by bo...@apache.org.
added STORM-634 to Changelog


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

Branch: refs/heads/master
Commit: bb8d48da29f0b93fe09f91dc3769517d0fdc045f
Parents: 66274f3
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Mar 18 14:37:51 2015 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Mar 18 14:37:51 2015 -0500

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


http://git-wip-us.apache.org/repos/asf/storm/blob/bb8d48da/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index f82cd55..3a76053 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -84,6 +84,7 @@
  * STORM-696: Single Namespace Test Launching
  * STORM-689: SimpleACLAuthorizer should provide a way to restrict who can submit topologies.
  * STORM-541: Build produces maven warnings
+ * STORM-634: Storm serialization changed to thrift to support rolling upgrade.
 
 ## 0.9.3-rc2
  * STORM-558: change "swap!" to "reset!" to fix assignment-versions in supervisor


[11/16] storm git commit: Regenrated the files using thrift-0.9.2 and fixed some bugs introduced by upmerging.

Posted by bo...@apache.org.
Regenrated the files using thrift-0.9.2 and fixed some bugs introduced by upmerging.


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

Branch: refs/heads/master
Commit: 539a6a21464acada9d051b3d5ce67990ffa2a239
Parents: 5b67b57
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Mar 5 17:21:54 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Mar 5 22:01:14 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/config.clj    |   2 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   2 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |   1 -
 .../backtype/storm/generated/Assignment.java    | 626 ++++++++------
 .../backtype/storm/generated/ExecutorStats.java | 105 ++-
 .../jvm/backtype/storm/generated/NodeInfo.java  | 245 ++++--
 .../jvm/backtype/storm/generated/StormBase.java | 577 ++++++++-----
 .../storm/generated/SupervisorInfo.java         | 648 +++++++++------
 .../storm/generated/TopologyActionOptions.java  |  76 +-
 .../storm/generated/TopologyStatus.java         |   3 +-
 .../storm/generated/ZKWorkerHeartbeat.java      | 307 ++++---
 storm-core/src/py/storm/ttypes.py               | 827 ++++++++++++++++++-
 12 files changed, 2505 insertions(+), 914 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index 5e7893b..a345daf 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -212,7 +212,7 @@
   (let [stormroot (supervisor-stormdist-root conf storm-id)
         conf-path (supervisor-stormconf-path stormroot)
         topology-path (supervisor-stormcode-path stormroot)]
-    (merge conf (clojurify-structure (Utils/javaDeserialize (FileUtils/readFileToByteArray (File. conf-path)))))))
+    (merge conf (clojurify-structure (Utils/javaDeserialize (FileUtils/readFileToByteArray (File. conf-path)) java.util.Map)))))
 
 (defn read-supervisor-topology
   [conf storm-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/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 2c7b82c..fdc54cc 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -19,7 +19,7 @@
   (:import [org.apache.thrift.exception])
   (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
   (:import [org.apache.commons.io FileUtils])
-  (:import [java.nio ByteBuffer])
+  (:import [java.nio ByteBuffer]
            [java.util Collections HashMap])
   (:import [java.io FileNotFoundException File FileOutputStream])
   (:import [java.nio.channels Channels WritableByteChannel])

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 88b14d8..e0263d6 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -25,7 +25,6 @@
   (:import [backtype.storm.utils Utils TransferDrainer ThriftTopologyUtils])
   (:import [backtype.storm.messaging TransportFactory])
   (:import [backtype.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
-  (:import [backtype.storm.daemon.common WorkerHeartbeat])
   (:import [backtype.storm.daemon Shutdownable])
   (:import [backtype.storm.serialization KryoTupleSerializer])
   (:import [backtype.storm.generated StormTopology])

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/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 b6ade94..2e6e8a3 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Assignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
@@ -16,13 +16,24 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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.commons.lang.builder.HashCodeBuilder;
+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;
@@ -35,10 +46,13 @@ 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;
 
-public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable {
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable, Comparable<Assignment> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment");
 
   private static final org.apache.thrift.protocol.TField MASTER_CODE_DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("master_code_dir", org.apache.thrift.protocol.TType.STRING, (short)1);
@@ -46,10 +60,16 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
   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 Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AssignmentStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AssignmentTupleSchemeFactory());
+  }
+
   private String master_code_dir; // required
-  private Map<String,String> node_host; // required
-  private Map<List<Long>,NodeInfo> executor_node_port; // required
-  private Map<List<Long>,Long> executor_start_time_secs; // required
+  private Map<String,String> node_host; // optional
+  private Map<List<Long>,NodeInfo> executor_node_port; // optional
+  private Map<List<Long>,Long> executor_start_time_secs; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -119,25 +139,25 @@ 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};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.MASTER_CODE_DIR, new org.apache.thrift.meta_data.FieldMetaData("master_code_dir", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.MASTER_CODE_DIR, new org.apache.thrift.meta_data.FieldMetaData("master_code_dir", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.NODE_HOST, new org.apache.thrift.meta_data.FieldMetaData("node_host", org.apache.thrift.TFieldRequirementType.OPTIONAL,
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+    tmpMap.put(_Fields.NODE_HOST, new org.apache.thrift.meta_data.FieldMetaData("node_host", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-    tmpMap.put(_Fields.EXECUTOR_NODE_PORT, new org.apache.thrift.meta_data.FieldMetaData("executor_node_port", org.apache.thrift.TFieldRequirementType.OPTIONAL,
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
-                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)),
+    tmpMap.put(_Fields.EXECUTOR_NODE_PORT, new org.apache.thrift.meta_data.FieldMetaData("executor_node_port", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)), 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NodeInfo.class))));
-    tmpMap.put(_Fields.EXECUTOR_START_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("executor_start_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL,
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
-                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)),
+    tmpMap.put(_Fields.EXECUTOR_START_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("executor_start_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Assignment.class, metaDataMap);
@@ -167,31 +187,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       this.master_code_dir = other.master_code_dir;
     }
     if (other.is_set_node_host()) {
-      Map<String,String> __this__node_host = new HashMap<String,String>();
-      for (Map.Entry<String, String> other_element : other.node_host.entrySet()) {
-
-        String other_element_key = other_element.getKey();
-        String other_element_value = other_element.getValue();
-
-        String __this__node_host_copy_key = other_element_key;
-
-        String __this__node_host_copy_value = other_element_value;
-
-        __this__node_host.put(__this__node_host_copy_key, __this__node_host_copy_value);
-      }
+      Map<String,String> __this__node_host = new HashMap<String,String>(other.node_host);
       this.node_host = __this__node_host;
     }
     if (other.is_set_executor_node_port()) {
-      Map<List<Long>,NodeInfo> __this__executor_node_port = new HashMap<List<Long>,NodeInfo>();
+      Map<List<Long>,NodeInfo> __this__executor_node_port = new HashMap<List<Long>,NodeInfo>(other.executor_node_port.size());
       for (Map.Entry<List<Long>, NodeInfo> other_element : other.executor_node_port.entrySet()) {
 
         List<Long> other_element_key = other_element.getKey();
         NodeInfo other_element_value = other_element.getValue();
 
-        List<Long> __this__executor_node_port_copy_key = new ArrayList<Long>();
-        for (Long other_element_key_element : other_element_key) {
-          __this__executor_node_port_copy_key.add(other_element_key_element);
-        }
+        List<Long> __this__executor_node_port_copy_key = new ArrayList<Long>(other_element_key);
 
         NodeInfo __this__executor_node_port_copy_value = new NodeInfo(other_element_value);
 
@@ -200,16 +206,13 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       this.executor_node_port = __this__executor_node_port;
     }
     if (other.is_set_executor_start_time_secs()) {
-      Map<List<Long>,Long> __this__executor_start_time_secs = new HashMap<List<Long>,Long>();
+      Map<List<Long>,Long> __this__executor_start_time_secs = new HashMap<List<Long>,Long>(other.executor_start_time_secs.size());
       for (Map.Entry<List<Long>, Long> other_element : other.executor_start_time_secs.entrySet()) {
 
         List<Long> other_element_key = other_element.getKey();
         Long other_element_value = other_element.getValue();
 
-        List<Long> __this__executor_start_time_secs_copy_key = new ArrayList<Long>();
-        for (Long other_element_key_element : other_element_key) {
-          __this__executor_start_time_secs_copy_key.add(other_element_key_element);
-        }
+        List<Long> __this__executor_start_time_secs_copy_key = new ArrayList<Long>(other_element_key);
 
         Long __this__executor_start_time_secs_copy_value = other_element_value;
 
@@ -487,75 +490,75 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
+    List<Object> list = new ArrayList<Object>();
 
     boolean present_master_code_dir = true && (is_set_master_code_dir());
-    builder.append(present_master_code_dir);
+    list.add(present_master_code_dir);
     if (present_master_code_dir)
-      builder.append(master_code_dir);
+      list.add(master_code_dir);
 
     boolean present_node_host = true && (is_set_node_host());
-    builder.append(present_node_host);
+    list.add(present_node_host);
     if (present_node_host)
-      builder.append(node_host);
+      list.add(node_host);
 
     boolean present_executor_node_port = true && (is_set_executor_node_port());
-    builder.append(present_executor_node_port);
+    list.add(present_executor_node_port);
     if (present_executor_node_port)
-      builder.append(executor_node_port);
+      list.add(executor_node_port);
 
     boolean present_executor_start_time_secs = true && (is_set_executor_start_time_secs());
-    builder.append(present_executor_start_time_secs);
+    list.add(present_executor_start_time_secs);
     if (present_executor_start_time_secs)
-      builder.append(executor_start_time_secs);
+      list.add(executor_start_time_secs);
 
-    return builder.toHashCode();
+    return list.hashCode();
   }
 
+  @Override
   public int compareTo(Assignment other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
 
     int lastComparison = 0;
-    Assignment typedOther = (Assignment)other;
 
-    lastComparison = Boolean.valueOf(is_set_master_code_dir()).compareTo(typedOther.is_set_master_code_dir());
+    lastComparison = Boolean.valueOf(is_set_master_code_dir()).compareTo(other.is_set_master_code_dir());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_master_code_dir()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.master_code_dir, typedOther.master_code_dir);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.master_code_dir, other.master_code_dir);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_node_host()).compareTo(typedOther.is_set_node_host());
+    lastComparison = Boolean.valueOf(is_set_node_host()).compareTo(other.is_set_node_host());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_node_host()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node_host, typedOther.node_host);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node_host, other.node_host);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_executor_node_port()).compareTo(typedOther.is_set_executor_node_port());
+    lastComparison = Boolean.valueOf(is_set_executor_node_port()).compareTo(other.is_set_executor_node_port());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_executor_node_port()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_node_port, typedOther.executor_node_port);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_node_port, other.executor_node_port);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_executor_start_time_secs()).compareTo(typedOther.is_set_executor_start_time_secs());
+    lastComparison = Boolean.valueOf(is_set_executor_start_time_secs()).compareTo(other.is_set_executor_start_time_secs());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_executor_start_time_secs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_start_time_secs, typedOther.executor_start_time_secs);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_start_time_secs, other.executor_start_time_secs);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -568,179 +571,11 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) {
-        break;
-      }
-      switch (field.id) {
-        case 1: // MASTER_CODE_DIR
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.master_code_dir = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // NODE_HOST
-          if (field.type == org.apache.thrift.protocol.TType.MAP) {
-            {
-              org.apache.thrift.protocol.TMap _map185 = iprot.readMapBegin();
-              this.node_host = new HashMap<String,String>(2*_map185.size);
-              for (int _i186 = 0; _i186 < _map185.size; ++_i186)
-              {
-                String _key187; // required
-                String _val188; // required
-                _key187 = iprot.readString();
-                _val188 = iprot.readString();
-                this.node_host.put(_key187, _val188);
-              }
-              iprot.readMapEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 3: // EXECUTOR_NODE_PORT
-          if (field.type == org.apache.thrift.protocol.TType.MAP) {
-            {
-              org.apache.thrift.protocol.TMap _map189 = iprot.readMapBegin();
-              this.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map189.size);
-              for (int _i190 = 0; _i190 < _map189.size; ++_i190)
-              {
-                List<Long> _key191; // required
-                NodeInfo _val192; // required
-                {
-                  org.apache.thrift.protocol.TList _list193 = iprot.readListBegin();
-                  _key191 = new ArrayList<Long>(_list193.size);
-                  for (int _i194 = 0; _i194 < _list193.size; ++_i194)
-                  {
-                    long _elem195; // required
-                    _elem195 = iprot.readI64();
-                    _key191.add(_elem195);
-                  }
-                  iprot.readListEnd();
-                }
-                _val192 = new NodeInfo();
-                _val192.read(iprot);
-                this.executor_node_port.put(_key191, _val192);
-              }
-              iprot.readMapEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 4: // EXECUTOR_START_TIME_SECS
-          if (field.type == org.apache.thrift.protocol.TType.MAP) {
-            {
-              org.apache.thrift.protocol.TMap _map196 = iprot.readMapBegin();
-              this.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map196.size);
-              for (int _i197 = 0; _i197 < _map196.size; ++_i197)
-              {
-                List<Long> _key198; // required
-                long _val199; // required
-                {
-                  org.apache.thrift.protocol.TList _list200 = iprot.readListBegin();
-                  _key198 = new ArrayList<Long>(_list200.size);
-                  for (int _i201 = 0; _i201 < _list200.size; ++_i201)
-                  {
-                    long _elem202; // required
-                    _elem202 = iprot.readI64();
-                    _key198.add(_elem202);
-                  }
-                  iprot.readListEnd();
-                }
-                _val199 = iprot.readI64();
-                this.executor_start_time_secs.put(_key198, _val199);
-              }
-              iprot.readMapEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    if (this.master_code_dir != null) {
-      oprot.writeFieldBegin(MASTER_CODE_DIR_FIELD_DESC);
-      oprot.writeString(this.master_code_dir);
-      oprot.writeFieldEnd();
-    }
-    if (this.node_host != null) {
-      if (is_set_node_host()) {
-        oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
-        {
-          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, this.node_host.size()));
-          for (Map.Entry<String, String> _iter203 : this.node_host.entrySet())
-          {
-            oprot.writeString(_iter203.getKey());
-            oprot.writeString(_iter203.getValue());
-          }
-          oprot.writeMapEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-    }
-    if (this.executor_node_port != null) {
-      if (is_set_executor_node_port()) {
-        oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
-        {
-          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, this.executor_node_port.size()));
-          for (Map.Entry<List<Long>, NodeInfo> _iter204 : this.executor_node_port.entrySet())
-          {
-            {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter204.getKey().size()));
-              for (long _iter205 : _iter204.getKey())
-              {
-                oprot.writeI64(_iter205);
-              }
-              oprot.writeListEnd();
-            }
-            _iter204.getValue().write(oprot);
-          }
-          oprot.writeMapEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-    }
-    if (this.executor_start_time_secs != null) {
-      if (is_set_executor_start_time_secs()) {
-        oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC);
-        {
-          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, this.executor_start_time_secs.size()));
-          for (Map.Entry<List<Long>, Long> _iter206 : this.executor_start_time_secs.entrySet())
-          {
-            {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter206.getKey().size()));
-              for (long _iter207 : _iter206.getKey())
-              {
-                oprot.writeI64(_iter207);
-              }
-              oprot.writeListEnd();
-            }
-            oprot.writeI64(_iter206.getValue());
-          }
-          oprot.writeMapEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-    }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -795,6 +630,7 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'master_code_dir' is unset! Struct:" + toString());
     }
 
+    // check for sub-struct validity
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -813,5 +649,335 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
     }
   }
 
+  private static class AssignmentStandardSchemeFactory implements SchemeFactory {
+    public AssignmentStandardScheme getScheme() {
+      return new AssignmentStandardScheme();
+    }
+  }
+
+  private static class AssignmentStandardScheme extends StandardScheme<Assignment> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MASTER_CODE_DIR
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.master_code_dir = iprot.readString();
+              struct.set_master_code_dir_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NODE_HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map370 = iprot.readMapBegin();
+                struct.node_host = new HashMap<String,String>(2*_map370.size);
+                String _key371;
+                String _val372;
+                for (int _i373 = 0; _i373 < _map370.size; ++_i373)
+                {
+                  _key371 = iprot.readString();
+                  _val372 = iprot.readString();
+                  struct.node_host.put(_key371, _val372);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_node_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EXECUTOR_NODE_PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map374 = iprot.readMapBegin();
+                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map374.size);
+                List<Long> _key375;
+                NodeInfo _val376;
+                for (int _i377 = 0; _i377 < _map374.size; ++_i377)
+                {
+                  {
+                    org.apache.thrift.protocol.TList _list378 = iprot.readListBegin();
+                    _key375 = new ArrayList<Long>(_list378.size);
+                    long _elem379;
+                    for (int _i380 = 0; _i380 < _list378.size; ++_i380)
+                    {
+                      _elem379 = iprot.readI64();
+                      _key375.add(_elem379);
+                    }
+                    iprot.readListEnd();
+                  }
+                  _val376 = new NodeInfo();
+                  _val376.read(iprot);
+                  struct.executor_node_port.put(_key375, _val376);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executor_node_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // EXECUTOR_START_TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map381 = iprot.readMapBegin();
+                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map381.size);
+                List<Long> _key382;
+                long _val383;
+                for (int _i384 = 0; _i384 < _map381.size; ++_i384)
+                {
+                  {
+                    org.apache.thrift.protocol.TList _list385 = iprot.readListBegin();
+                    _key382 = new ArrayList<Long>(_list385.size);
+                    long _elem386;
+                    for (int _i387 = 0; _i387 < _list385.size; ++_i387)
+                    {
+                      _elem386 = iprot.readI64();
+                      _key382.add(_elem386);
+                    }
+                    iprot.readListEnd();
+                  }
+                  _val383 = iprot.readI64();
+                  struct.executor_start_time_secs.put(_key382, _val383);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executor_start_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.master_code_dir != null) {
+        oprot.writeFieldBegin(MASTER_CODE_DIR_FIELD_DESC);
+        oprot.writeString(struct.master_code_dir);
+        oprot.writeFieldEnd();
+      }
+      if (struct.node_host != null) {
+        if (struct.is_set_node_host()) {
+          oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.node_host.size()));
+            for (Map.Entry<String, String> _iter388 : struct.node_host.entrySet())
+            {
+              oprot.writeString(_iter388.getKey());
+              oprot.writeString(_iter388.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.executor_node_port != null) {
+        if (struct.is_set_executor_node_port()) {
+          oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, struct.executor_node_port.size()));
+            for (Map.Entry<List<Long>, NodeInfo> _iter389 : struct.executor_node_port.entrySet())
+            {
+              {
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter389.getKey().size()));
+                for (long _iter390 : _iter389.getKey())
+                {
+                  oprot.writeI64(_iter390);
+                }
+                oprot.writeListEnd();
+              }
+              _iter389.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.executor_start_time_secs != null) {
+        if (struct.is_set_executor_start_time_secs()) {
+          oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, struct.executor_start_time_secs.size()));
+            for (Map.Entry<List<Long>, Long> _iter391 : struct.executor_start_time_secs.entrySet())
+            {
+              {
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter391.getKey().size()));
+                for (long _iter392 : _iter391.getKey())
+                {
+                  oprot.writeI64(_iter392);
+                }
+                oprot.writeListEnd();
+              }
+              oprot.writeI64(_iter391.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AssignmentTupleSchemeFactory implements SchemeFactory {
+    public AssignmentTupleScheme getScheme() {
+      return new AssignmentTupleScheme();
+    }
+  }
+
+  private static class AssignmentTupleScheme extends TupleScheme<Assignment> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, Assignment struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.master_code_dir);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_node_host()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_executor_node_port()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_executor_start_time_secs()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_node_host()) {
+        {
+          oprot.writeI32(struct.node_host.size());
+          for (Map.Entry<String, String> _iter393 : struct.node_host.entrySet())
+          {
+            oprot.writeString(_iter393.getKey());
+            oprot.writeString(_iter393.getValue());
+          }
+        }
+      }
+      if (struct.is_set_executor_node_port()) {
+        {
+          oprot.writeI32(struct.executor_node_port.size());
+          for (Map.Entry<List<Long>, NodeInfo> _iter394 : struct.executor_node_port.entrySet())
+          {
+            {
+              oprot.writeI32(_iter394.getKey().size());
+              for (long _iter395 : _iter394.getKey())
+              {
+                oprot.writeI64(_iter395);
+              }
+            }
+            _iter394.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> _iter396 : struct.executor_start_time_secs.entrySet())
+          {
+            {
+              oprot.writeI32(_iter396.getKey().size());
+              for (long _iter397 : _iter396.getKey())
+              {
+                oprot.writeI64(_iter397);
+              }
+            }
+            oprot.writeI64(_iter396.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, Assignment struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.master_code_dir = iprot.readString();
+      struct.set_master_code_dir_isSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map398 = 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*_map398.size);
+          String _key399;
+          String _val400;
+          for (int _i401 = 0; _i401 < _map398.size; ++_i401)
+          {
+            _key399 = iprot.readString();
+            _val400 = iprot.readString();
+            struct.node_host.put(_key399, _val400);
+          }
+        }
+        struct.set_node_host_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map402 = 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*_map402.size);
+          List<Long> _key403;
+          NodeInfo _val404;
+          for (int _i405 = 0; _i405 < _map402.size; ++_i405)
+          {
+            {
+              org.apache.thrift.protocol.TList _list406 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key403 = new ArrayList<Long>(_list406.size);
+              long _elem407;
+              for (int _i408 = 0; _i408 < _list406.size; ++_i408)
+              {
+                _elem407 = iprot.readI64();
+                _key403.add(_elem407);
+              }
+            }
+            _val404 = new NodeInfo();
+            _val404.read(iprot);
+            struct.executor_node_port.put(_key403, _val404);
+          }
+        }
+        struct.set_executor_node_port_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TMap _map409 = 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*_map409.size);
+          List<Long> _key410;
+          long _val411;
+          for (int _i412 = 0; _i412 < _map409.size; ++_i412)
+          {
+            {
+              org.apache.thrift.protocol.TList _list413 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key410 = new ArrayList<Long>(_list413.size);
+              long _elem414;
+              for (int _i415 = 0; _i415 < _list413.size; ++_i415)
+              {
+                _elem414 = iprot.readI64();
+                _key410.add(_elem414);
+              }
+            }
+            _val411 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key410, _val411);
+          }
+        }
+        struct.set_executor_start_time_secs_isSet(true);
+      }
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/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 d82a9cf..001fcff 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
@@ -51,13 +51,14 @@ 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-3-5")
 public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, ExecutorStats._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorStats");
 
   private static final org.apache.thrift.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("emitted", org.apache.thrift.protocol.TType.MAP, (short)1);
   private static final org.apache.thrift.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("transferred", org.apache.thrift.protocol.TType.MAP, (short)2);
   private static final org.apache.thrift.protocol.TField SPECIFIC_FIELD_DESC = new org.apache.thrift.protocol.TField("specific", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField RATE_FIELD_DESC = new org.apache.thrift.protocol.TField("rate", org.apache.thrift.protocol.TType.DOUBLE, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -68,12 +69,14 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
   private Map<String,Map<String,Long>> emitted; // required
   private Map<String,Map<String,Long>> transferred; // required
   private ExecutorSpecificStats specific; // required
+  private double rate; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     EMITTED((short)1, "emitted"),
     TRANSFERRED((short)2, "transferred"),
-    SPECIFIC((short)3, "specific");
+    SPECIFIC((short)3, "specific"),
+    RATE((short)4, "rate");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -94,6 +97,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
           return TRANSFERRED;
         case 3: // SPECIFIC
           return SPECIFIC;
+        case 4: // RATE
+          return RATE;
         default:
           return null;
       }
@@ -134,6 +139,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
   }
 
   // isset id assignments
+  private static final int __RATE_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -151,6 +158,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
                 new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
     tmpMap.put(_Fields.SPECIFIC, new org.apache.thrift.meta_data.FieldMetaData("specific", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSpecificStats.class)));
+    tmpMap.put(_Fields.RATE, new org.apache.thrift.meta_data.FieldMetaData("rate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorStats.class, metaDataMap);
   }
@@ -161,18 +170,22 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
   public ExecutorStats(
     Map<String,Map<String,Long>> emitted,
     Map<String,Map<String,Long>> transferred,
-    ExecutorSpecificStats specific)
+    ExecutorSpecificStats specific,
+    double rate)
   {
     this();
     this.emitted = emitted;
     this.transferred = transferred;
     this.specific = specific;
+    this.rate = rate;
+    set_rate_isSet(true);
   }
 
   /**
    * Performs a deep copy on <i>other</i>.
    */
   public ExecutorStats(ExecutorStats other) {
+    __isset_bitfield = other.__isset_bitfield;
     if (other.is_set_emitted()) {
       Map<String,Map<String,Long>> __this__emitted = new HashMap<String,Map<String,Long>>(other.emitted.size());
       for (Map.Entry<String, Map<String,Long>> other_element : other.emitted.entrySet()) {
@@ -206,6 +219,7 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     if (other.is_set_specific()) {
       this.specific = new ExecutorSpecificStats(other.specific);
     }
+    this.rate = other.rate;
   }
 
   public ExecutorStats deepCopy() {
@@ -217,6 +231,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     this.emitted = null;
     this.transferred = null;
     this.specific = null;
+    set_rate_isSet(false);
+    this.rate = 0.0;
   }
 
   public int get_emitted_size() {
@@ -310,6 +326,28 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     }
   }
 
+  public double get_rate() {
+    return this.rate;
+  }
+
+  public void set_rate(double rate) {
+    this.rate = rate;
+    set_rate_isSet(true);
+  }
+
+  public void unset_rate() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RATE_ISSET_ID);
+  }
+
+  /** Returns true if field rate is set (has been assigned a value) and false otherwise */
+  public boolean is_set_rate() {
+    return EncodingUtils.testBit(__isset_bitfield, __RATE_ISSET_ID);
+  }
+
+  public void set_rate_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RATE_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case EMITTED:
@@ -336,6 +374,14 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       }
       break;
 
+    case RATE:
+      if (value == null) {
+        unset_rate();
+      } else {
+        set_rate((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -350,6 +396,9 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     case SPECIFIC:
       return get_specific();
 
+    case RATE:
+      return Double.valueOf(get_rate());
+
     }
     throw new IllegalStateException();
   }
@@ -367,6 +416,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       return is_set_transferred();
     case SPECIFIC:
       return is_set_specific();
+    case RATE:
+      return is_set_rate();
     }
     throw new IllegalStateException();
   }
@@ -411,6 +462,15 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         return false;
     }
 
+    boolean this_present_rate = true;
+    boolean that_present_rate = true;
+    if (this_present_rate || that_present_rate) {
+      if (!(this_present_rate && that_present_rate))
+        return false;
+      if (this.rate != that.rate)
+        return false;
+    }
+
     return true;
   }
 
@@ -433,6 +493,11 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     if (present_specific)
       list.add(specific);
 
+    boolean present_rate = true;
+    list.add(present_rate);
+    if (present_rate)
+      list.add(rate);
+
     return list.hashCode();
   }
 
@@ -474,6 +539,16 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_rate()).compareTo(other.is_set_rate());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_rate()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rate, other.rate);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -517,6 +592,10 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       sb.append(this.specific);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("rate:");
+    sb.append(this.rate);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -535,6 +614,10 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'specific' is unset! Struct:" + toString());
     }
 
+    if (!is_set_rate()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'rate' is unset! Struct:" + toString());
+    }
+
     // check for sub-struct validity
   }
 
@@ -548,6 +631,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
 
   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);
@@ -645,6 +730,14 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // RATE
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.rate = iprot.readDouble();
+              struct.set_rate_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -705,6 +798,9 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         struct.specific.write(oprot);
         oprot.writeFieldEnd();
       }
+      oprot.writeFieldBegin(RATE_FIELD_DESC);
+      oprot.writeDouble(struct.rate);
+      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -753,6 +849,7 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         }
       }
       struct.specific.write(oprot);
+      oprot.writeDouble(struct.rate);
     }
 
     @Override
@@ -809,6 +906,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       struct.specific = new ExecutorSpecificStats();
       struct.specific.read(iprot);
       struct.set_specific_isSet(true);
+      struct.rate = iprot.readDouble();
+      struct.set_rate_isSet(true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/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 f9b46e9..a71d1b4 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
@@ -16,13 +16,24 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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.commons.lang.builder.HashCodeBuilder;
+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;
@@ -35,15 +46,24 @@ 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;
 
-public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable {
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<NodeInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo");
 
   private static final org.apache.thrift.protocol.TField NODE_FIELD_DESC = new org.apache.thrift.protocol.TField("node", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.SET, (short)2);
 
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NodeInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NodeInfoTupleSchemeFactory());
+  }
+
   private String node; // required
   private Set<Long> port; // required
 
@@ -109,14 +129,13 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
   }
 
   // isset id assignments
-
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.NODE, new org.apache.thrift.meta_data.FieldMetaData("node", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.NODE, new org.apache.thrift.meta_data.FieldMetaData("node", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED,
-        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET,
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NodeInfo.class, metaDataMap);
@@ -142,10 +161,7 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       this.node = other.node;
     }
     if (other.is_set_port()) {
-      Set<Long> __this__port = new HashSet<Long>();
-      for (Long other_element : other.port) {
-        __this__port.add(other_element);
-      }
+      Set<Long> __this__port = new HashSet<Long>(other.port);
       this.port = __this__port;
     }
   }
@@ -305,45 +321,45 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
+    List<Object> list = new ArrayList<Object>();
 
     boolean present_node = true && (is_set_node());
-    builder.append(present_node);
+    list.add(present_node);
     if (present_node)
-      builder.append(node);
+      list.add(node);
 
     boolean present_port = true && (is_set_port());
-    builder.append(present_port);
+    list.add(present_port);
     if (present_port)
-      builder.append(port);
+      list.add(port);
 
-    return builder.toHashCode();
+    return list.hashCode();
   }
 
+  @Override
   public int compareTo(NodeInfo other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
 
     int lastComparison = 0;
-    NodeInfo typedOther = (NodeInfo)other;
 
-    lastComparison = Boolean.valueOf(is_set_node()).compareTo(typedOther.is_set_node());
+    lastComparison = Boolean.valueOf(is_set_node()).compareTo(other.is_set_node());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_node()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node, typedOther.node);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node, other.node);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port());
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_port()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, typedOther.port);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -356,71 +372,11 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) {
-        break;
-      }
-      switch (field.id) {
-        case 1: // NODE
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.node = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // PORT
-          if (field.type == org.apache.thrift.protocol.TType.SET) {
-            {
-              org.apache.thrift.protocol.TSet _set181 = iprot.readSetBegin();
-              this.port = new HashSet<Long>(2*_set181.size);
-              for (int _i182 = 0; _i182 < _set181.size; ++_i182)
-              {
-                long _elem183; // required
-                _elem183 = iprot.readI64();
-                this.port.add(_elem183);
-              }
-              iprot.readSetEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    if (this.node != null) {
-      oprot.writeFieldBegin(NODE_FIELD_DESC);
-      oprot.writeString(this.node);
-      oprot.writeFieldEnd();
-    }
-    if (this.port != null) {
-      oprot.writeFieldBegin(PORT_FIELD_DESC);
-      {
-        oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, this.port.size()));
-        for (long _iter184 : this.port)
-        {
-          oprot.writeI64(_iter184);
-        }
-        oprot.writeSetEnd();
-      }
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -457,6 +413,7 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
     }
 
+    // check for sub-struct validity
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -475,5 +432,125 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
     }
   }
 
+  private static class NodeInfoStandardSchemeFactory implements SchemeFactory {
+    public NodeInfoStandardScheme getScheme() {
+      return new NodeInfoStandardScheme();
+    }
+  }
+
+  private static class NodeInfoStandardScheme extends StandardScheme<NodeInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NodeInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NODE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.node = iprot.readString();
+              struct.set_node_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
+              {
+                org.apache.thrift.protocol.TSet _set362 = iprot.readSetBegin();
+                struct.port = new HashSet<Long>(2*_set362.size);
+                long _elem363;
+                for (int _i364 = 0; _i364 < _set362.size; ++_i364)
+                {
+                  _elem363 = iprot.readI64();
+                  struct.port.add(_elem363);
+                }
+                iprot.readSetEnd();
+              }
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, NodeInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.node != null) {
+        oprot.writeFieldBegin(NODE_FIELD_DESC);
+        oprot.writeString(struct.node);
+        oprot.writeFieldEnd();
+      }
+      if (struct.port != null) {
+        oprot.writeFieldBegin(PORT_FIELD_DESC);
+        {
+          oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size()));
+          for (long _iter365 : struct.port)
+          {
+            oprot.writeI64(_iter365);
+          }
+          oprot.writeSetEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NodeInfoTupleSchemeFactory implements SchemeFactory {
+    public NodeInfoTupleScheme getScheme() {
+      return new NodeInfoTupleScheme();
+    }
+  }
+
+  private static class NodeInfoTupleScheme extends TupleScheme<NodeInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.node);
+      {
+        oprot.writeI32(struct.port.size());
+        for (long _iter366 : struct.port)
+        {
+          oprot.writeI64(_iter366);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.node = iprot.readString();
+      struct.set_node_isSet(true);
+      {
+        org.apache.thrift.protocol.TSet _set367 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.port = new HashSet<Long>(2*_set367.size);
+        long _elem368;
+        for (int _i369 = 0; _i369 < _set367.size; ++_i369)
+        {
+          _elem368 = iprot.readI64();
+          struct.port.add(_elem368);
+        }
+      }
+      struct.set_port_isSet(true);
+    }
+  }
+
 }
 


[09/16] storm git commit: Regenrated the files using thrift-0.9.2 and fixed some bugs introduced by upmerging.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
index 47d8428..01c5c67 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ZKWorkerHeartbeat.java
@@ -16,13 +16,24 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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.commons.lang.builder.HashCodeBuilder;
+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;
@@ -35,16 +46,25 @@ 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;
 
-public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartbeat, ZKWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable {
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartbeat, ZKWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ZKWorkerHeartbeat> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ZKWorkerHeartbeat");
 
   private static final org.apache.thrift.protocol.TField STORM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storm_id", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField EXECUTOR_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_stats", org.apache.thrift.protocol.TType.MAP, (short)2);
   private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)3);
 
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ZKWorkerHeartbeatStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ZKWorkerHeartbeatTupleSchemeFactory());
+  }
+
   private String storm_id; // required
   private Map<ExecutorInfo,ExecutorStats> executor_stats; // required
   private int time_secs; // required
@@ -115,18 +135,17 @@ public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartb
 
   // isset id assignments
   private static final int __TIME_SECS_ISSET_ID = 0;
-  private BitSet __isset_bit_vector = new BitSet(1);
-
+  private byte __isset_bitfield = 0;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.STORM_ID, new org.apache.thrift.meta_data.FieldMetaData("storm_id", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.STORM_ID, new org.apache.thrift.meta_data.FieldMetaData("storm_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.EXECUTOR_STATS, new org.apache.thrift.meta_data.FieldMetaData("executor_stats", org.apache.thrift.TFieldRequirementType.REQUIRED,
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class),
+    tmpMap.put(_Fields.EXECUTOR_STATS, new org.apache.thrift.meta_data.FieldMetaData("executor_stats", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class), 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorStats.class))));
-    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED,
+    tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ZKWorkerHeartbeat.class, metaDataMap);
@@ -151,13 +170,12 @@ public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartb
    * Performs a deep copy on <i>other</i>.
    */
   public ZKWorkerHeartbeat(ZKWorkerHeartbeat other) {
-    __isset_bit_vector.clear();
-    __isset_bit_vector.or(other.__isset_bit_vector);
+    __isset_bitfield = other.__isset_bitfield;
     if (other.is_set_storm_id()) {
       this.storm_id = other.storm_id;
     }
     if (other.is_set_executor_stats()) {
-      Map<ExecutorInfo,ExecutorStats> __this__executor_stats = new HashMap<ExecutorInfo,ExecutorStats>();
+      Map<ExecutorInfo,ExecutorStats> __this__executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(other.executor_stats.size());
       for (Map.Entry<ExecutorInfo, ExecutorStats> other_element : other.executor_stats.entrySet()) {
 
         ExecutorInfo other_element_key = other_element.getKey();
@@ -253,16 +271,16 @@ public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartb
   }
 
   public void unset_time_secs() {
-    __isset_bit_vector.clear(__TIME_SECS_ISSET_ID);
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
   }
 
   /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */
   public boolean is_set_time_secs() {
-    return __isset_bit_vector.get(__TIME_SECS_ISSET_ID);
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID);
   }
 
   public void set_time_secs_isSet(boolean value) {
-    __isset_bit_vector.set(__TIME_SECS_ISSET_ID, value);
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value);
   }
 
   public void setFieldValue(_Fields field, Object value) {
@@ -371,60 +389,60 @@ public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartb
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
+    List<Object> list = new ArrayList<Object>();
 
     boolean present_storm_id = true && (is_set_storm_id());
-    builder.append(present_storm_id);
+    list.add(present_storm_id);
     if (present_storm_id)
-      builder.append(storm_id);
+      list.add(storm_id);
 
     boolean present_executor_stats = true && (is_set_executor_stats());
-    builder.append(present_executor_stats);
+    list.add(present_executor_stats);
     if (present_executor_stats)
-      builder.append(executor_stats);
+      list.add(executor_stats);
 
     boolean present_time_secs = true;
-    builder.append(present_time_secs);
+    list.add(present_time_secs);
     if (present_time_secs)
-      builder.append(time_secs);
+      list.add(time_secs);
 
-    return builder.toHashCode();
+    return list.hashCode();
   }
 
+  @Override
   public int compareTo(ZKWorkerHeartbeat other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
 
     int lastComparison = 0;
-    ZKWorkerHeartbeat typedOther = (ZKWorkerHeartbeat)other;
 
-    lastComparison = Boolean.valueOf(is_set_storm_id()).compareTo(typedOther.is_set_storm_id());
+    lastComparison = Boolean.valueOf(is_set_storm_id()).compareTo(other.is_set_storm_id());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_storm_id()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_id, typedOther.storm_id);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_id, other.storm_id);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_executor_stats()).compareTo(typedOther.is_set_executor_stats());
+    lastComparison = Boolean.valueOf(is_set_executor_stats()).compareTo(other.is_set_executor_stats());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_executor_stats()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_stats, typedOther.executor_stats);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_stats, other.executor_stats);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(typedOther.is_set_time_secs());
+    lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_time_secs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, typedOther.time_secs);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -437,87 +455,11 @@ public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartb
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) {
-        break;
-      }
-      switch (field.id) {
-        case 1: // STORM_ID
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.storm_id = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // EXECUTOR_STATS
-          if (field.type == org.apache.thrift.protocol.TType.MAP) {
-            {
-              org.apache.thrift.protocol.TMap _map213 = iprot.readMapBegin();
-              this.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map213.size);
-              for (int _i214 = 0; _i214 < _map213.size; ++_i214)
-              {
-                ExecutorInfo _key215; // required
-                ExecutorStats _val216; // required
-                _key215 = new ExecutorInfo();
-                _key215.read(iprot);
-                _val216 = new ExecutorStats();
-                _val216.read(iprot);
-                this.executor_stats.put(_key215, _val216);
-              }
-              iprot.readMapEnd();
-            }
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 3: // TIME_SECS
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.time_secs = iprot.readI32();
-            set_time_secs_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    if (this.storm_id != null) {
-      oprot.writeFieldBegin(STORM_ID_FIELD_DESC);
-      oprot.writeString(this.storm_id);
-      oprot.writeFieldEnd();
-    }
-    if (this.executor_stats != null) {
-      oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
-      {
-        oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, this.executor_stats.size()));
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter217 : this.executor_stats.entrySet())
-        {
-          _iter217.getKey().write(oprot);
-          _iter217.getValue().write(oprot);
-        }
-        oprot.writeMapEnd();
-      }
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
-    oprot.writeI32(this.time_secs);
-    oprot.writeFieldEnd();
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -562,6 +504,7 @@ public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartb
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString());
     }
 
+    // check for sub-struct validity
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -575,12 +518,156 @@ public class ZKWorkerHeartbeat implements org.apache.thrift.TBase<ZKWorkerHeartb
   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_bit_vector = new BitSet(1);
+      __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 ZKWorkerHeartbeatStandardSchemeFactory implements SchemeFactory {
+    public ZKWorkerHeartbeatStandardScheme getScheme() {
+      return new ZKWorkerHeartbeatStandardScheme();
+    }
+  }
+
+  private static class ZKWorkerHeartbeatStandardScheme extends StandardScheme<ZKWorkerHeartbeat> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // STORM_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.storm_id = iprot.readString();
+              struct.set_storm_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // EXECUTOR_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map426.size);
+                ExecutorInfo _key427;
+                ExecutorStats _val428;
+                for (int _i429 = 0; _i429 < _map426.size; ++_i429)
+                {
+                  _key427 = new ExecutorInfo();
+                  _key427.read(iprot);
+                  _val428 = new ExecutorStats();
+                  _val428.read(iprot);
+                  struct.executor_stats.put(_key427, _val428);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_executor_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.time_secs = iprot.readI32();
+              struct.set_time_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          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, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.storm_id != null) {
+        oprot.writeFieldBegin(STORM_ID_FIELD_DESC);
+        oprot.writeString(struct.storm_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.executor_stats != null) {
+        oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter430 : struct.executor_stats.entrySet())
+          {
+            _iter430.getKey().write(oprot);
+            _iter430.getValue().write(oprot);
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.time_secs);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ZKWorkerHeartbeatTupleSchemeFactory implements SchemeFactory {
+    public ZKWorkerHeartbeatTupleScheme getScheme() {
+      return new ZKWorkerHeartbeatTupleScheme();
+    }
+  }
+
+  private static class ZKWorkerHeartbeatTupleScheme extends TupleScheme<ZKWorkerHeartbeat> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.storm_id);
+      {
+        oprot.writeI32(struct.executor_stats.size());
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter431 : struct.executor_stats.entrySet())
+        {
+          _iter431.getKey().write(oprot);
+          _iter431.getValue().write(oprot);
+        }
+      }
+      oprot.writeI32(struct.time_secs);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ZKWorkerHeartbeat struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.storm_id = iprot.readString();
+      struct.set_storm_id_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map432 = 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*_map432.size);
+        ExecutorInfo _key433;
+        ExecutorStats _val434;
+        for (int _i435 = 0; _i435 < _map432.size; ++_i435)
+        {
+          _key433 = new ExecutorInfo();
+          _key433.read(iprot);
+          _val434 = new ExecutorStats();
+          _val434.read(iprot);
+          struct.executor_stats.put(_key433, _val434);
+        }
+      }
+      struct.set_executor_stats_isSet(true);
+      struct.time_secs = iprot.readI32();
+      struct.set_time_secs_isSet(true);
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/539a6a21/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 8c3f5e6..924f761 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -46,6 +46,26 @@ class TopologyInitialStatus:
     "INACTIVE": 2,
   }
 
+class TopologyStatus:
+  ACTIVE = 1
+  INACTIVE = 2
+  REBALANCING = 3
+  KILLED = 4
+
+  _VALUES_TO_NAMES = {
+    1: "ACTIVE",
+    2: "INACTIVE",
+    3: "REBALANCING",
+    4: "KILLED",
+  }
+
+  _NAMES_TO_VALUES = {
+    "ACTIVE": 1,
+    "INACTIVE": 2,
+    "REBALANCING": 3,
+    "KILLED": 4,
+  }
+
 class NumErrorsChoice:
   ALL = 0
   NONE = 1
@@ -3150,6 +3170,7 @@ class ExecutorStats:
    - emitted
    - transferred
    - specific
+   - rate
   """
 
   thrift_spec = (
@@ -3157,12 +3178,14 @@ class ExecutorStats:
     (1, TType.MAP, 'emitted', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 1
     (2, TType.MAP, 'transferred', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 2
     (3, TType.STRUCT, 'specific', (ExecutorSpecificStats, ExecutorSpecificStats.thrift_spec), None, ), # 3
+    (4, TType.DOUBLE, 'rate', None, None, ), # 4
   )
 
-  def __init__(self, emitted=None, transferred=None, specific=None,):
+  def __init__(self, emitted=None, transferred=None, specific=None, rate=None,):
     self.emitted = emitted
     self.transferred = transferred
     self.specific = specific
+    self.rate = rate
 
   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:
@@ -3213,6 +3236,11 @@ class ExecutorStats:
           self.specific.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.DOUBLE:
+          self.rate = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3251,6 +3279,10 @@ class ExecutorStats:
       oprot.writeFieldBegin('specific', TType.STRUCT, 3)
       self.specific.write(oprot)
       oprot.writeFieldEnd()
+    if self.rate is not None:
+      oprot.writeFieldBegin('rate', TType.DOUBLE, 4)
+      oprot.writeDouble(self.rate)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3261,6 +3293,8 @@ class ExecutorStats:
       raise TProtocol.TProtocolException(message='Required field transferred is unset!')
     if self.specific is None:
       raise TProtocol.TProtocolException(message='Required field specific is unset!')
+    if self.rate is None:
+      raise TProtocol.TProtocolException(message='Required field rate is unset!')
     return
 
 
@@ -3269,6 +3303,7 @@ class ExecutorStats:
     value = (value * 31) ^ hash(self.emitted)
     value = (value * 31) ^ hash(self.transferred)
     value = (value * 31) ^ hash(self.specific)
+    value = (value * 31) ^ hash(self.rate)
     return value
 
   def __repr__(self):
@@ -4533,6 +4568,796 @@ class SubmitOptions:
   def __ne__(self, other):
     return not (self == other)
 
+class SupervisorInfo:
+  """
+  Attributes:
+   - time_secs
+   - hostname
+   - assignment_id
+   - used_ports
+   - meta
+   - scheduler_meta
+   - uptime_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'time_secs', None, None, ), # 1
+    (2, TType.STRING, 'hostname', None, None, ), # 2
+    (3, TType.STRING, 'assignment_id', None, None, ), # 3
+    (4, TType.LIST, 'used_ports', (TType.I64,None), None, ), # 4
+    (5, TType.LIST, 'meta', (TType.I64,None), None, ), # 5
+    (6, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 6
+    (7, TType.I64, 'uptime_secs', None, None, ), # 7
+  )
+
+  def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None,):
+    self.time_secs = time_secs
+    self.hostname = hostname
+    self.assignment_id = assignment_id
+    self.used_ports = used_ports
+    self.meta = meta
+    self.scheduler_meta = scheduler_meta
+    self.uptime_secs = uptime_secs
+
+  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.I64:
+          self.time_secs = iprot.readI64();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.hostname = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.assignment_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.used_ports = []
+          (_etype304, _size301) = iprot.readListBegin()
+          for _i305 in xrange(_size301):
+            _elem306 = iprot.readI64();
+            self.used_ports.append(_elem306)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.LIST:
+          self.meta = []
+          (_etype310, _size307) = iprot.readListBegin()
+          for _i311 in xrange(_size307):
+            _elem312 = iprot.readI64();
+            self.meta.append(_elem312)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.MAP:
+          self.scheduler_meta = {}
+          (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin()
+          for _i317 in xrange(_size313):
+            _key318 = iprot.readString().decode('utf-8')
+            _val319 = iprot.readString().decode('utf-8')
+            self.scheduler_meta[_key318] = _val319
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I64:
+          self.uptime_secs = iprot.readI64();
+        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('SupervisorInfo')
+    if self.time_secs is not None:
+      oprot.writeFieldBegin('time_secs', TType.I64, 1)
+      oprot.writeI64(self.time_secs)
+      oprot.writeFieldEnd()
+    if self.hostname is not None:
+      oprot.writeFieldBegin('hostname', TType.STRING, 2)
+      oprot.writeString(self.hostname.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.assignment_id is not None:
+      oprot.writeFieldBegin('assignment_id', TType.STRING, 3)
+      oprot.writeString(self.assignment_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.used_ports is not None:
+      oprot.writeFieldBegin('used_ports', TType.LIST, 4)
+      oprot.writeListBegin(TType.I64, len(self.used_ports))
+      for iter320 in self.used_ports:
+        oprot.writeI64(iter320)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.meta is not None:
+      oprot.writeFieldBegin('meta', TType.LIST, 5)
+      oprot.writeListBegin(TType.I64, len(self.meta))
+      for iter321 in self.meta:
+        oprot.writeI64(iter321)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.scheduler_meta is not None:
+      oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta))
+      for kiter322,viter323 in self.scheduler_meta.items():
+        oprot.writeString(kiter322.encode('utf-8'))
+        oprot.writeString(viter323.encode('utf-8'))
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.uptime_secs is not None:
+      oprot.writeFieldBegin('uptime_secs', TType.I64, 7)
+      oprot.writeI64(self.uptime_secs)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.time_secs is None:
+      raise TProtocol.TProtocolException(message='Required field time_secs is unset!')
+    if self.hostname is None:
+      raise TProtocol.TProtocolException(message='Required field hostname is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.time_secs)
+    value = (value * 31) ^ hash(self.hostname)
+    value = (value * 31) ^ hash(self.assignment_id)
+    value = (value * 31) ^ hash(self.used_ports)
+    value = (value * 31) ^ hash(self.meta)
+    value = (value * 31) ^ hash(self.scheduler_meta)
+    value = (value * 31) ^ hash(self.uptime_secs)
+    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 NodeInfo:
+  """
+  Attributes:
+   - node
+   - port
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'node', None, None, ), # 1
+    (2, TType.SET, 'port', (TType.I64,None), None, ), # 2
+  )
+
+  def __init__(self, node=None, port=None,):
+    self.node = node
+    self.port = port
+
+  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.STRING:
+          self.node = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.SET:
+          self.port = set()
+          (_etype327, _size324) = iprot.readSetBegin()
+          for _i328 in xrange(_size324):
+            _elem329 = iprot.readI64();
+            self.port.add(_elem329)
+          iprot.readSetEnd()
+        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('NodeInfo')
+    if self.node is not None:
+      oprot.writeFieldBegin('node', TType.STRING, 1)
+      oprot.writeString(self.node.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.port is not None:
+      oprot.writeFieldBegin('port', TType.SET, 2)
+      oprot.writeSetBegin(TType.I64, len(self.port))
+      for iter330 in self.port:
+        oprot.writeI64(iter330)
+      oprot.writeSetEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.node is None:
+      raise TProtocol.TProtocolException(message='Required field node is unset!')
+    if self.port is None:
+      raise TProtocol.TProtocolException(message='Required field port is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.node)
+    value = (value * 31) ^ hash(self.port)
+    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:
+   - master_code_dir
+   - node_host
+   - executor_node_port
+   - executor_start_time_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'master_code_dir', None, None, ), # 1
+    (2, TType.MAP, 'node_host', (TType.STRING,None,TType.STRING,None), {
+    }, ), # 2
+    (3, TType.MAP, 'executor_node_port', (TType.LIST,(TType.I64,None),TType.STRUCT,(NodeInfo, NodeInfo.thrift_spec)), {
+    }, ), # 3
+    (4, TType.MAP, 'executor_start_time_secs', (TType.LIST,(TType.I64,None),TType.I64,None), {
+    }, ), # 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],):
+    self.master_code_dir = master_code_dir
+    if node_host is self.thrift_spec[2][4]:
+      node_host = {
+    }
+    self.node_host = node_host
+    if executor_node_port is self.thrift_spec[3][4]:
+      executor_node_port = {
+    }
+    self.executor_node_port = executor_node_port
+    if executor_start_time_secs is self.thrift_spec[4][4]:
+      executor_start_time_secs = {
+    }
+    self.executor_start_time_secs = executor_start_time_secs
+
+  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.STRING:
+          self.master_code_dir = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.node_host = {}
+          (_ktype332, _vtype333, _size331 ) = iprot.readMapBegin()
+          for _i335 in xrange(_size331):
+            _key336 = iprot.readString().decode('utf-8')
+            _val337 = iprot.readString().decode('utf-8')
+            self.node_host[_key336] = _val337
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.MAP:
+          self.executor_node_port = {}
+          (_ktype339, _vtype340, _size338 ) = iprot.readMapBegin()
+          for _i342 in xrange(_size338):
+            _key343 = []
+            (_etype348, _size345) = iprot.readListBegin()
+            for _i349 in xrange(_size345):
+              _elem350 = iprot.readI64();
+              _key343.append(_elem350)
+            iprot.readListEnd()
+            _val344 = NodeInfo()
+            _val344.read(iprot)
+            self.executor_node_port[_key343] = _val344
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.MAP:
+          self.executor_start_time_secs = {}
+          (_ktype352, _vtype353, _size351 ) = iprot.readMapBegin()
+          for _i355 in xrange(_size351):
+            _key356 = []
+            (_etype361, _size358) = iprot.readListBegin()
+            for _i362 in xrange(_size358):
+              _elem363 = iprot.readI64();
+              _key356.append(_elem363)
+            iprot.readListEnd()
+            _val357 = iprot.readI64();
+            self.executor_start_time_secs[_key356] = _val357
+          iprot.readMapEnd()
+        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('Assignment')
+    if self.master_code_dir is not None:
+      oprot.writeFieldBegin('master_code_dir', TType.STRING, 1)
+      oprot.writeString(self.master_code_dir.encode('utf-8'))
+      oprot.writeFieldEnd()
+    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 kiter364,viter365 in self.node_host.items():
+        oprot.writeString(kiter364.encode('utf-8'))
+        oprot.writeString(viter365.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 kiter366,viter367 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter366))
+        for iter368 in kiter366:
+          oprot.writeI64(iter368)
+        oprot.writeListEnd()
+        viter367.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 kiter369,viter370 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter369))
+        for iter371 in kiter369:
+          oprot.writeI64(iter371)
+        oprot.writeListEnd()
+        oprot.writeI64(viter370)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.master_code_dir is None:
+      raise TProtocol.TProtocolException(message='Required field master_code_dir is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.master_code_dir)
+    value = (value * 31) ^ hash(self.node_host)
+    value = (value * 31) ^ hash(self.executor_node_port)
+    value = (value * 31) ^ hash(self.executor_start_time_secs)
+    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 TopologyActionOptions:
+  """
+  Attributes:
+   - kill_options
+   - rebalance_options
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'kill_options', (KillOptions, KillOptions.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'rebalance_options', (RebalanceOptions, RebalanceOptions.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, kill_options=None, rebalance_options=None,):
+    self.kill_options = kill_options
+    self.rebalance_options = rebalance_options
+
+  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.STRUCT:
+          self.kill_options = KillOptions()
+          self.kill_options.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.rebalance_options = RebalanceOptions()
+          self.rebalance_options.read(iprot)
+        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('TopologyActionOptions')
+    if self.kill_options is not None:
+      oprot.writeFieldBegin('kill_options', TType.STRUCT, 1)
+      self.kill_options.write(oprot)
+      oprot.writeFieldEnd()
+    if self.rebalance_options is not None:
+      oprot.writeFieldBegin('rebalance_options', TType.STRUCT, 2)
+      self.rebalance_options.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.kill_options)
+    value = (value * 31) ^ hash(self.rebalance_options)
+    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 StormBase:
+  """
+  Attributes:
+   - name
+   - status
+   - num_workers
+   - component_executors
+   - launch_time_secs
+   - owner
+   - topology_action_options
+   - prev_status
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.I32, 'status', None, None, ), # 2
+    (3, TType.I32, 'num_workers', None, None, ), # 3
+    (4, TType.MAP, 'component_executors', (TType.STRING,None,TType.I32,None), None, ), # 4
+    (5, TType.I32, 'launch_time_secs', None, None, ), # 5
+    (6, TType.STRING, 'owner', None, None, ), # 6
+    (7, TType.STRUCT, 'topology_action_options', (TopologyActionOptions, TopologyActionOptions.thrift_spec), None, ), # 7
+    (8, TType.I32, 'prev_status', None, None, ), # 8
+  )
+
+  def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None,):
+    self.name = name
+    self.status = status
+    self.num_workers = num_workers
+    self.component_executors = component_executors
+    self.launch_time_secs = launch_time_secs
+    self.owner = owner
+    self.topology_action_options = topology_action_options
+    self.prev_status = prev_status
+
+  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.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.status = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.num_workers = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.MAP:
+          self.component_executors = {}
+          (_ktype373, _vtype374, _size372 ) = iprot.readMapBegin()
+          for _i376 in xrange(_size372):
+            _key377 = iprot.readString().decode('utf-8')
+            _val378 = iprot.readI32();
+            self.component_executors[_key377] = _val378
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.launch_time_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.owner = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRUCT:
+          self.topology_action_options = TopologyActionOptions()
+          self.topology_action_options.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.I32:
+          self.prev_status = iprot.readI32();
+        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('StormBase')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.status is not None:
+      oprot.writeFieldBegin('status', TType.I32, 2)
+      oprot.writeI32(self.status)
+      oprot.writeFieldEnd()
+    if self.num_workers is not None:
+      oprot.writeFieldBegin('num_workers', TType.I32, 3)
+      oprot.writeI32(self.num_workers)
+      oprot.writeFieldEnd()
+    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 kiter379,viter380 in self.component_executors.items():
+        oprot.writeString(kiter379.encode('utf-8'))
+        oprot.writeI32(viter380)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.launch_time_secs is not None:
+      oprot.writeFieldBegin('launch_time_secs', TType.I32, 5)
+      oprot.writeI32(self.launch_time_secs)
+      oprot.writeFieldEnd()
+    if self.owner is not None:
+      oprot.writeFieldBegin('owner', TType.STRING, 6)
+      oprot.writeString(self.owner.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.topology_action_options is not None:
+      oprot.writeFieldBegin('topology_action_options', TType.STRUCT, 7)
+      self.topology_action_options.write(oprot)
+      oprot.writeFieldEnd()
+    if self.prev_status is not None:
+      oprot.writeFieldBegin('prev_status', TType.I32, 8)
+      oprot.writeI32(self.prev_status)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.name is None:
+      raise TProtocol.TProtocolException(message='Required field name is unset!')
+    if self.status is None:
+      raise TProtocol.TProtocolException(message='Required field status is unset!')
+    if self.num_workers is None:
+      raise TProtocol.TProtocolException(message='Required field num_workers is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.status)
+    value = (value * 31) ^ hash(self.num_workers)
+    value = (value * 31) ^ hash(self.component_executors)
+    value = (value * 31) ^ hash(self.launch_time_secs)
+    value = (value * 31) ^ hash(self.owner)
+    value = (value * 31) ^ hash(self.topology_action_options)
+    value = (value * 31) ^ hash(self.prev_status)
+    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 ZKWorkerHeartbeat:
+  """
+  Attributes:
+   - storm_id
+   - executor_stats
+   - time_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'storm_id', None, None, ), # 1
+    (2, TType.MAP, 'executor_stats', (TType.STRUCT,(ExecutorInfo, ExecutorInfo.thrift_spec),TType.STRUCT,(ExecutorStats, ExecutorStats.thrift_spec)), None, ), # 2
+    (3, TType.I32, 'time_secs', None, None, ), # 3
+  )
+
+  def __init__(self, storm_id=None, executor_stats=None, time_secs=None,):
+    self.storm_id = storm_id
+    self.executor_stats = executor_stats
+    self.time_secs = time_secs
+
+  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.STRING:
+          self.storm_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.executor_stats = {}
+          (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin()
+          for _i385 in xrange(_size381):
+            _key386 = ExecutorInfo()
+            _key386.read(iprot)
+            _val387 = ExecutorStats()
+            _val387.read(iprot)
+            self.executor_stats[_key386] = _val387
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.time_secs = iprot.readI32();
+        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('ZKWorkerHeartbeat')
+    if self.storm_id is not None:
+      oprot.writeFieldBegin('storm_id', TType.STRING, 1)
+      oprot.writeString(self.storm_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    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 kiter388,viter389 in self.executor_stats.items():
+        kiter388.write(oprot)
+        viter389.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.time_secs is not None:
+      oprot.writeFieldBegin('time_secs', TType.I32, 3)
+      oprot.writeI32(self.time_secs)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.storm_id is None:
+      raise TProtocol.TProtocolException(message='Required field storm_id is unset!')
+    if self.executor_stats is None:
+      raise TProtocol.TProtocolException(message='Required field executor_stats is unset!')
+    if self.time_secs is None:
+      raise TProtocol.TProtocolException(message='Required field time_secs is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.storm_id)
+    value = (value * 31) ^ hash(self.executor_stats)
+    value = (value * 31) ^ hash(self.time_secs)
+    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 GetInfoOptions:
   """
   Attributes:


[03/16] storm git commit: STORM-634: Converting SupervisorInfo, Assignment, StormBase, TopologyStatus, ZKWorkerHeartbeat, ErrorInfo, Credentials to thrift and defaulting the serialization delegate to thrift serialization. Added class as a param to serializatio

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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 a297fa7..15d629e 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
@@ -44,16 +44,19 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
   private static final org.apache.thrift.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("emitted", org.apache.thrift.protocol.TType.MAP, (short)1);
   private static final org.apache.thrift.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("transferred", org.apache.thrift.protocol.TType.MAP, (short)2);
   private static final org.apache.thrift.protocol.TField SPECIFIC_FIELD_DESC = new org.apache.thrift.protocol.TField("specific", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField RATE_FIELD_DESC = new org.apache.thrift.protocol.TField("rate", org.apache.thrift.protocol.TType.DOUBLE, (short)4);
 
   private Map<String,Map<String,Long>> emitted; // required
   private Map<String,Map<String,Long>> transferred; // required
   private ExecutorSpecificStats specific; // required
+  private double rate; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     EMITTED((short)1, "emitted"),
     TRANSFERRED((short)2, "transferred"),
-    SPECIFIC((short)3, "specific");
+    SPECIFIC((short)3, "specific"),
+    RATE((short)4, "rate");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -74,6 +77,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
           return TRANSFERRED;
         case 3: // SPECIFIC
           return SPECIFIC;
+        case 4: // RATE
+          return RATE;
         default:
           return null;
       }
@@ -114,24 +119,28 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
   }
 
   // isset id assignments
+  private static final int __RATE_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.EMITTED, new org.apache.thrift.meta_data.FieldMetaData("emitted", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+    tmpMap.put(_Fields.EMITTED, new org.apache.thrift.meta_data.FieldMetaData("emitted", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
                 new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
-    tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("transferred", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+    tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("transferred", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+            new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+                new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
                 new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))));
-    tmpMap.put(_Fields.SPECIFIC, new org.apache.thrift.meta_data.FieldMetaData("specific", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+    tmpMap.put(_Fields.SPECIFIC, new org.apache.thrift.meta_data.FieldMetaData("specific", org.apache.thrift.TFieldRequirementType.REQUIRED,
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSpecificStats.class)));
+    tmpMap.put(_Fields.RATE, new org.apache.thrift.meta_data.FieldMetaData("rate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorStats.class, metaDataMap);
   }
@@ -142,18 +151,23 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
   public ExecutorStats(
     Map<String,Map<String,Long>> emitted,
     Map<String,Map<String,Long>> transferred,
-    ExecutorSpecificStats specific)
+    ExecutorSpecificStats specific,
+    double rate)
   {
     this();
     this.emitted = emitted;
     this.transferred = transferred;
     this.specific = specific;
+    this.rate = rate;
+    set_rate_isSet(true);
   }
 
   /**
    * Performs a deep copy on <i>other</i>.
    */
   public ExecutorStats(ExecutorStats other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
     if (other.is_set_emitted()) {
       Map<String,Map<String,Long>> __this__emitted = new HashMap<String,Map<String,Long>>();
       for (Map.Entry<String, Map<String,Long>> other_element : other.emitted.entrySet()) {
@@ -209,6 +223,7 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     if (other.is_set_specific()) {
       this.specific = new ExecutorSpecificStats(other.specific);
     }
+    this.rate = other.rate;
   }
 
   public ExecutorStats deepCopy() {
@@ -220,6 +235,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     this.emitted = null;
     this.transferred = null;
     this.specific = null;
+    set_rate_isSet(false);
+    this.rate = 0.0;
   }
 
   public int get_emitted_size() {
@@ -313,6 +330,28 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     }
   }
 
+  public double get_rate() {
+    return this.rate;
+  }
+
+  public void set_rate(double rate) {
+    this.rate = rate;
+    set_rate_isSet(true);
+  }
+
+  public void unset_rate() {
+    __isset_bit_vector.clear(__RATE_ISSET_ID);
+  }
+
+  /** Returns true if field rate is set (has been assigned a value) and false otherwise */
+  public boolean is_set_rate() {
+    return __isset_bit_vector.get(__RATE_ISSET_ID);
+  }
+
+  public void set_rate_isSet(boolean value) {
+    __isset_bit_vector.set(__RATE_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case EMITTED:
@@ -339,6 +378,14 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       }
       break;
 
+    case RATE:
+      if (value == null) {
+        unset_rate();
+      } else {
+        set_rate((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -353,6 +400,9 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     case SPECIFIC:
       return get_specific();
 
+    case RATE:
+      return Double.valueOf(get_rate());
+
     }
     throw new IllegalStateException();
   }
@@ -370,6 +420,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       return is_set_transferred();
     case SPECIFIC:
       return is_set_specific();
+    case RATE:
+      return is_set_rate();
     }
     throw new IllegalStateException();
   }
@@ -414,6 +466,15 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         return false;
     }
 
+    boolean this_present_rate = true;
+    boolean that_present_rate = true;
+    if (this_present_rate || that_present_rate) {
+      if (!(this_present_rate && that_present_rate))
+        return false;
+      if (this.rate != that.rate)
+        return false;
+    }
+
     return true;
   }
 
@@ -436,6 +497,11 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     if (present_specific)
       builder.append(specific);
 
+    boolean present_rate = true;
+    builder.append(present_rate);
+    if (present_rate)
+      builder.append(rate);
+
     return builder.toHashCode();
   }
 
@@ -477,6 +543,16 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_rate()).compareTo(typedOther.is_set_rate());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_rate()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rate, typedOther.rate);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -490,7 +566,7 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     while (true)
     {
       field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+      if (field.type == org.apache.thrift.protocol.TType.STOP) {
         break;
       }
       switch (field.id) {
@@ -564,6 +640,14 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 4: // RATE
+          if (field.type == org.apache.thrift.protocol.TType.DOUBLE) {
+            this.rate = iprot.readDouble();
+            set_rate_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
       }
@@ -624,6 +708,9 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       this.specific.write(oprot);
       oprot.writeFieldEnd();
     }
+    oprot.writeFieldBegin(RATE_FIELD_DESC);
+    oprot.writeDouble(this.rate);
+    oprot.writeFieldEnd();
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
@@ -656,6 +743,10 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       sb.append(this.specific);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("rate:");
+    sb.append(this.rate);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -674,6 +765,10 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'specific' is unset! Struct:" + toString());
     }
 
+    if (!is_set_rate()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'rate' is unset! Struct:" + toString());
+    }
+
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -686,6 +781,8 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
 
   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_bit_vector = new BitSet(1);
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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
new file mode 100644
index 0000000..f9b46e9
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
@@ -0,0 +1,479 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo");
+
+  private static final org.apache.thrift.protocol.TField NODE_FIELD_DESC = new org.apache.thrift.protocol.TField("node", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.SET, (short)2);
+
+  private String node; // required
+  private Set<Long> port; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NODE((short)1, "node"),
+    PORT((short)2, "port");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NODE
+          return NODE;
+        case 2: // PORT
+          return PORT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NODE, new org.apache.thrift.meta_data.FieldMetaData("node", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NodeInfo.class, metaDataMap);
+  }
+
+  public NodeInfo() {
+  }
+
+  public NodeInfo(
+    String node,
+    Set<Long> port)
+  {
+    this();
+    this.node = node;
+    this.port = port;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NodeInfo(NodeInfo other) {
+    if (other.is_set_node()) {
+      this.node = other.node;
+    }
+    if (other.is_set_port()) {
+      Set<Long> __this__port = new HashSet<Long>();
+      for (Long other_element : other.port) {
+        __this__port.add(other_element);
+      }
+      this.port = __this__port;
+    }
+  }
+
+  public NodeInfo deepCopy() {
+    return new NodeInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.node = null;
+    this.port = null;
+  }
+
+  public String get_node() {
+    return this.node;
+  }
+
+  public void set_node(String node) {
+    this.node = node;
+  }
+
+  public void unset_node() {
+    this.node = null;
+  }
+
+  /** Returns true if field node is set (has been assigned a value) and false otherwise */
+  public boolean is_set_node() {
+    return this.node != null;
+  }
+
+  public void set_node_isSet(boolean value) {
+    if (!value) {
+      this.node = null;
+    }
+  }
+
+  public int get_port_size() {
+    return (this.port == null) ? 0 : this.port.size();
+  }
+
+  public java.util.Iterator<Long> get_port_iterator() {
+    return (this.port == null) ? null : this.port.iterator();
+  }
+
+  public void add_to_port(long elem) {
+    if (this.port == null) {
+      this.port = new HashSet<Long>();
+    }
+    this.port.add(elem);
+  }
+
+  public Set<Long> get_port() {
+    return this.port;
+  }
+
+  public void set_port(Set<Long> port) {
+    this.port = port;
+  }
+
+  public void unset_port() {
+    this.port = null;
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return this.port != null;
+  }
+
+  public void set_port_isSet(boolean value) {
+    if (!value) {
+      this.port = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NODE:
+      if (value == null) {
+        unset_node();
+      } else {
+        set_node((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Set<Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NODE:
+      return get_node();
+
+    case PORT:
+      return get_port();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NODE:
+      return is_set_node();
+    case PORT:
+      return is_set_port();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NodeInfo)
+      return this.equals((NodeInfo)that);
+    return false;
+  }
+
+  public boolean equals(NodeInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_node = true && this.is_set_node();
+    boolean that_present_node = true && that.is_set_node();
+    if (this_present_node || that_present_node) {
+      if (!(this_present_node && that_present_node))
+        return false;
+      if (!this.node.equals(that.node))
+        return false;
+    }
+
+    boolean this_present_port = true && this.is_set_port();
+    boolean that_present_port = true && that.is_set_port();
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (!this.port.equals(that.port))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_node = true && (is_set_node());
+    builder.append(present_node);
+    if (present_node)
+      builder.append(node);
+
+    boolean present_port = true && (is_set_port());
+    builder.append(present_port);
+    if (present_port)
+      builder.append(port);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(NodeInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    NodeInfo typedOther = (NodeInfo)other;
+
+    lastComparison = Boolean.valueOf(is_set_node()).compareTo(typedOther.is_set_node());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_node()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node, typedOther.node);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, typedOther.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        break;
+      }
+      switch (field.id) {
+        case 1: // NODE
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.node = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // PORT
+          if (field.type == org.apache.thrift.protocol.TType.SET) {
+            {
+              org.apache.thrift.protocol.TSet _set181 = iprot.readSetBegin();
+              this.port = new HashSet<Long>(2*_set181.size);
+              for (int _i182 = 0; _i182 < _set181.size; ++_i182)
+              {
+                long _elem183; // required
+                _elem183 = iprot.readI64();
+                this.port.add(_elem183);
+              }
+              iprot.readSetEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.node != null) {
+      oprot.writeFieldBegin(NODE_FIELD_DESC);
+      oprot.writeString(this.node);
+      oprot.writeFieldEnd();
+    }
+    if (this.port != null) {
+      oprot.writeFieldBegin(PORT_FIELD_DESC);
+      {
+        oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, this.port.size()));
+        for (long _iter184 : this.port)
+        {
+          oprot.writeI64(_iter184);
+        }
+        oprot.writeSetEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NodeInfo(");
+    boolean first = true;
+
+    sb.append("node:");
+    if (this.node == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.node);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    if (this.port == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.port);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_node()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'node' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/63900643/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
new file mode 100644
index 0000000..e27d635
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
@@ -0,0 +1,1078 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField COMPONENT_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("component_executors", org.apache.thrift.protocol.TType.MAP, (short)4);
+  private static final org.apache.thrift.protocol.TField LAUNCH_TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("launch_time_secs", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ACTION_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_action_options", org.apache.thrift.protocol.TType.STRUCT, (short)7);
+  private static final org.apache.thrift.protocol.TField PREV_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("prev_status", org.apache.thrift.protocol.TType.I32, (short)8);
+
+  private String name; // required
+  private TopologyStatus status; // required
+  private int num_workers; // required
+  private Map<String,Integer> component_executors; // required
+  private int launch_time_secs; // required
+  private String owner; // required
+  private TopologyActionOptions topology_action_options; // required
+  private TopologyStatus prev_status; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    /**
+     * 
+     * @see TopologyStatus
+     */
+    STATUS((short)2, "status"),
+    NUM_WORKERS((short)3, "num_workers"),
+    COMPONENT_EXECUTORS((short)4, "component_executors"),
+    LAUNCH_TIME_SECS((short)5, "launch_time_secs"),
+    OWNER((short)6, "owner"),
+    TOPOLOGY_ACTION_OPTIONS((short)7, "topology_action_options"),
+    /**
+     * 
+     * @see TopologyStatus
+     */
+    PREV_STATUS((short)8, "prev_status");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // STATUS
+          return STATUS;
+        case 3: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 4: // COMPONENT_EXECUTORS
+          return COMPONENT_EXECUTORS;
+        case 5: // LAUNCH_TIME_SECS
+          return LAUNCH_TIME_SECS;
+        case 6: // OWNER
+          return OWNER;
+        case 7: // TOPOLOGY_ACTION_OPTIONS
+          return TOPOLOGY_ACTION_OPTIONS;
+        case 8: // PREV_STATUS
+          return PREV_STATUS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_WORKERS_ISSET_ID = 0;
+  private static final int __LAUNCH_TIME_SECS_ISSET_ID = 1;
+  private BitSet __isset_bit_vector = new BitSet(2);
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.COMPONENT_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("component_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
+    tmpMap.put(_Fields.LAUNCH_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("launch_time_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOPOLOGY_ACTION_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("topology_action_options", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyActionOptions.class)));
+    tmpMap.put(_Fields.PREV_STATUS, new org.apache.thrift.meta_data.FieldMetaData("prev_status", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormBase.class, metaDataMap);
+  }
+
+  public StormBase() {
+  }
+
+  public StormBase(
+    String name,
+    TopologyStatus status,
+    int num_workers)
+  {
+    this();
+    this.name = name;
+    this.status = status;
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StormBase(StormBase other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    if (other.is_set_status()) {
+      this.status = other.status;
+    }
+    this.num_workers = other.num_workers;
+    if (other.is_set_component_executors()) {
+      Map<String,Integer> __this__component_executors = new HashMap<String,Integer>();
+      for (Map.Entry<String, Integer> other_element : other.component_executors.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Integer other_element_value = other_element.getValue();
+
+        String __this__component_executors_copy_key = other_element_key;
+
+        Integer __this__component_executors_copy_value = other_element_value;
+
+        __this__component_executors.put(__this__component_executors_copy_key, __this__component_executors_copy_value);
+      }
+      this.component_executors = __this__component_executors;
+    }
+    this.launch_time_secs = other.launch_time_secs;
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
+    if (other.is_set_topology_action_options()) {
+      this.topology_action_options = new TopologyActionOptions(other.topology_action_options);
+    }
+    if (other.is_set_prev_status()) {
+      this.prev_status = other.prev_status;
+    }
+  }
+
+  public StormBase deepCopy() {
+    return new StormBase(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.status = null;
+    set_num_workers_isSet(false);
+    this.num_workers = 0;
+    this.component_executors = null;
+    set_launch_time_secs_isSet(false);
+    this.launch_time_secs = 0;
+    this.owner = null;
+    this.topology_action_options = null;
+    this.prev_status = null;
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public TopologyStatus get_status() {
+    return this.status;
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public void set_status(TopologyStatus status) {
+    this.status = status;
+  }
+
+  public void unset_status() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_status() {
+    return this.status != null;
+  }
+
+  public void set_status_isSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_component_executors_size() {
+    return (this.component_executors == null) ? 0 : this.component_executors.size();
+  }
+
+  public void put_to_component_executors(String key, int val) {
+    if (this.component_executors == null) {
+      this.component_executors = new HashMap<String,Integer>();
+    }
+    this.component_executors.put(key, val);
+  }
+
+  public Map<String,Integer> get_component_executors() {
+    return this.component_executors;
+  }
+
+  public void set_component_executors(Map<String,Integer> component_executors) {
+    this.component_executors = component_executors;
+  }
+
+  public void unset_component_executors() {
+    this.component_executors = null;
+  }
+
+  /** Returns true if field component_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_executors() {
+    return this.component_executors != null;
+  }
+
+  public void set_component_executors_isSet(boolean value) {
+    if (!value) {
+      this.component_executors = null;
+    }
+  }
+
+  public int get_launch_time_secs() {
+    return this.launch_time_secs;
+  }
+
+  public void set_launch_time_secs(int launch_time_secs) {
+    this.launch_time_secs = launch_time_secs;
+    set_launch_time_secs_isSet(true);
+  }
+
+  public void unset_launch_time_secs() {
+    __isset_bit_vector.clear(__LAUNCH_TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field launch_time_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_launch_time_secs() {
+    return __isset_bit_vector.get(__LAUNCH_TIME_SECS_ISSET_ID);
+  }
+
+  public void set_launch_time_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__LAUNCH_TIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = null;
+  }
+
+  /** Returns true if field owner is set (has been assigned a value) and false otherwise */
+  public boolean is_set_owner() {
+    return this.owner != null;
+  }
+
+  public void set_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
+  public TopologyActionOptions get_topology_action_options() {
+    return this.topology_action_options;
+  }
+
+  public void set_topology_action_options(TopologyActionOptions topology_action_options) {
+    this.topology_action_options = topology_action_options;
+  }
+
+  public void unset_topology_action_options() {
+    this.topology_action_options = null;
+  }
+
+  /** Returns true if field topology_action_options is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_action_options() {
+    return this.topology_action_options != null;
+  }
+
+  public void set_topology_action_options_isSet(boolean value) {
+    if (!value) {
+      this.topology_action_options = null;
+    }
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public TopologyStatus get_prev_status() {
+    return this.prev_status;
+  }
+
+  /**
+   * 
+   * @see TopologyStatus
+   */
+  public void set_prev_status(TopologyStatus prev_status) {
+    this.prev_status = prev_status;
+  }
+
+  public void unset_prev_status() {
+    this.prev_status = null;
+  }
+
+  /** Returns true if field prev_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_prev_status() {
+    return this.prev_status != null;
+  }
+
+  public void set_prev_status_isSet(boolean value) {
+    if (!value) {
+      this.prev_status = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unset_status();
+      } else {
+        set_status((TopologyStatus)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case COMPONENT_EXECUTORS:
+      if (value == null) {
+        unset_component_executors();
+      } else {
+        set_component_executors((Map<String,Integer>)value);
+      }
+      break;
+
+    case LAUNCH_TIME_SECS:
+      if (value == null) {
+        unset_launch_time_secs();
+      } else {
+        set_launch_time_secs((Integer)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
+    case TOPOLOGY_ACTION_OPTIONS:
+      if (value == null) {
+        unset_topology_action_options();
+      } else {
+        set_topology_action_options((TopologyActionOptions)value);
+      }
+      break;
+
+    case PREV_STATUS:
+      if (value == null) {
+        unset_prev_status();
+      } else {
+        set_prev_status((TopologyStatus)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return get_name();
+
+    case STATUS:
+      return get_status();
+
+    case NUM_WORKERS:
+      return Integer.valueOf(get_num_workers());
+
+    case COMPONENT_EXECUTORS:
+      return get_component_executors();
+
+    case LAUNCH_TIME_SECS:
+      return Integer.valueOf(get_launch_time_secs());
+
+    case OWNER:
+      return get_owner();
+
+    case TOPOLOGY_ACTION_OPTIONS:
+      return get_topology_action_options();
+
+    case PREV_STATUS:
+      return get_prev_status();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return is_set_name();
+    case STATUS:
+      return is_set_status();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case COMPONENT_EXECUTORS:
+      return is_set_component_executors();
+    case LAUNCH_TIME_SECS:
+      return is_set_launch_time_secs();
+    case OWNER:
+      return is_set_owner();
+    case TOPOLOGY_ACTION_OPTIONS:
+      return is_set_topology_action_options();
+    case PREV_STATUS:
+      return is_set_prev_status();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StormBase)
+      return this.equals((StormBase)that);
+    return false;
+  }
+
+  public boolean equals(StormBase that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_status = true && this.is_set_status();
+    boolean that_present_status = true && that.is_set_status();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    boolean this_present_num_workers = true;
+    boolean that_present_num_workers = true;
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_component_executors = true && this.is_set_component_executors();
+    boolean that_present_component_executors = true && that.is_set_component_executors();
+    if (this_present_component_executors || that_present_component_executors) {
+      if (!(this_present_component_executors && that_present_component_executors))
+        return false;
+      if (!this.component_executors.equals(that.component_executors))
+        return false;
+    }
+
+    boolean this_present_launch_time_secs = true && this.is_set_launch_time_secs();
+    boolean that_present_launch_time_secs = true && that.is_set_launch_time_secs();
+    if (this_present_launch_time_secs || that_present_launch_time_secs) {
+      if (!(this_present_launch_time_secs && that_present_launch_time_secs))
+        return false;
+      if (this.launch_time_secs != that.launch_time_secs)
+        return false;
+    }
+
+    boolean this_present_owner = true && this.is_set_owner();
+    boolean that_present_owner = true && that.is_set_owner();
+    if (this_present_owner || that_present_owner) {
+      if (!(this_present_owner && that_present_owner))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
+    boolean this_present_topology_action_options = true && this.is_set_topology_action_options();
+    boolean that_present_topology_action_options = true && that.is_set_topology_action_options();
+    if (this_present_topology_action_options || that_present_topology_action_options) {
+      if (!(this_present_topology_action_options && that_present_topology_action_options))
+        return false;
+      if (!this.topology_action_options.equals(that.topology_action_options))
+        return false;
+    }
+
+    boolean this_present_prev_status = true && this.is_set_prev_status();
+    boolean that_present_prev_status = true && that.is_set_prev_status();
+    if (this_present_prev_status || that_present_prev_status) {
+      if (!(this_present_prev_status && that_present_prev_status))
+        return false;
+      if (!this.prev_status.equals(that.prev_status))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_name = true && (is_set_name());
+    builder.append(present_name);
+    if (present_name)
+      builder.append(name);
+
+    boolean present_status = true && (is_set_status());
+    builder.append(present_status);
+    if (present_status)
+      builder.append(status.getValue());
+
+    boolean present_num_workers = true;
+    builder.append(present_num_workers);
+    if (present_num_workers)
+      builder.append(num_workers);
+
+    boolean present_component_executors = true && (is_set_component_executors());
+    builder.append(present_component_executors);
+    if (present_component_executors)
+      builder.append(component_executors);
+
+    boolean present_launch_time_secs = true && (is_set_launch_time_secs());
+    builder.append(present_launch_time_secs);
+    if (present_launch_time_secs)
+      builder.append(launch_time_secs);
+
+    boolean present_owner = true && (is_set_owner());
+    builder.append(present_owner);
+    if (present_owner)
+      builder.append(owner);
+
+    boolean present_topology_action_options = true && (is_set_topology_action_options());
+    builder.append(present_topology_action_options);
+    if (present_topology_action_options)
+      builder.append(topology_action_options);
+
+    boolean present_prev_status = true && (is_set_prev_status());
+    builder.append(present_prev_status);
+    if (present_prev_status)
+      builder.append(prev_status.getValue());
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(StormBase other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    StormBase typedOther = (StormBase)other;
+
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_status()).compareTo(typedOther.is_set_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_executors()).compareTo(typedOther.is_set_component_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_executors, typedOther.component_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_launch_time_secs()).compareTo(typedOther.is_set_launch_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_launch_time_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.launch_time_secs, typedOther.launch_time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(typedOther.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_action_options()).compareTo(typedOther.is_set_topology_action_options());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_action_options()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_action_options, typedOther.topology_action_options);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_prev_status()).compareTo(typedOther.is_set_prev_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_prev_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.prev_status, typedOther.prev_status);
+      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 {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        break;
+      }
+      switch (field.id) {
+        case 1: // NAME
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.name = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // STATUS
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.status = TopologyStatus.findByValue(iprot.readI32());
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // NUM_WORKERS
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.num_workers = iprot.readI32();
+            set_num_workers_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // COMPONENT_EXECUTORS
+          if (field.type == org.apache.thrift.protocol.TType.MAP) {
+            {
+              org.apache.thrift.protocol.TMap _map208 = iprot.readMapBegin();
+              this.component_executors = new HashMap<String,Integer>(2*_map208.size);
+              for (int _i209 = 0; _i209 < _map208.size; ++_i209)
+              {
+                String _key210; // required
+                int _val211; // required
+                _key210 = iprot.readString();
+                _val211 = iprot.readI32();
+                this.component_executors.put(_key210, _val211);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 5: // LAUNCH_TIME_SECS
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.launch_time_secs = iprot.readI32();
+            set_launch_time_secs_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 6: // OWNER
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.owner = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 7: // TOPOLOGY_ACTION_OPTIONS
+          if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+            this.topology_action_options = new TopologyActionOptions();
+            this.topology_action_options.read(iprot);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 8: // PREV_STATUS
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.prev_status = TopologyStatus.findByValue(iprot.readI32());
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.name != null) {
+      oprot.writeFieldBegin(NAME_FIELD_DESC);
+      oprot.writeString(this.name);
+      oprot.writeFieldEnd();
+    }
+    if (this.status != null) {
+      oprot.writeFieldBegin(STATUS_FIELD_DESC);
+      oprot.writeI32(this.status.getValue());
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+    oprot.writeI32(this.num_workers);
+    oprot.writeFieldEnd();
+    if (this.component_executors != null) {
+      if (is_set_component_executors()) {
+        oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, this.component_executors.size()));
+          for (Map.Entry<String, Integer> _iter212 : this.component_executors.entrySet())
+          {
+            oprot.writeString(_iter212.getKey());
+            oprot.writeI32(_iter212.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+    }
+    if (is_set_launch_time_secs()) {
+      oprot.writeFieldBegin(LAUNCH_TIME_SECS_FIELD_DESC);
+      oprot.writeI32(this.launch_time_secs);
+      oprot.writeFieldEnd();
+    }
+    if (this.owner != null) {
+      if (is_set_owner()) {
+        oprot.writeFieldBegin(OWNER_FIELD_DESC);
+        oprot.writeString(this.owner);
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.topology_action_options != null) {
+      if (is_set_topology_action_options()) {
+        oprot.writeFieldBegin(TOPOLOGY_ACTION_OPTIONS_FIELD_DESC);
+        this.topology_action_options.write(oprot);
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.prev_status != null) {
+      if (is_set_prev_status()) {
+        oprot.writeFieldBegin(PREV_STATUS_FIELD_DESC);
+        oprot.writeI32(this.prev_status.getValue());
+        oprot.writeFieldEnd();
+      }
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StormBase(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("status:");
+    if (this.status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.status);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_workers:");
+    sb.append(this.num_workers);
+    first = false;
+    if (is_set_component_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("component_executors:");
+      if (this.component_executors == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_executors);
+      }
+      first = false;
+    }
+    if (is_set_launch_time_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("launch_time_secs:");
+      sb.append(this.launch_time_secs);
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
+    if (is_set_topology_action_options()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_action_options:");
+      if (this.topology_action_options == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_action_options);
+      }
+      first = false;
+    }
+    if (is_set_prev_status()) {
+      if (!first) sb.append(", ");
+      sb.append("prev_status:");
+      if (this.prev_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.prev_status);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_status()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_workers()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString());
+    }
+
+  }
+
+  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_bit_vector = new BitSet(1);
+      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);
+    }
+  }
+
+}
+


[05/16] storm git commit: Merge remote-tracking branch 'upstream/master' into ru

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into ru

Conflicts:
	storm-core/src/clj/backtype/storm/cluster.clj
	storm-core/src/py/storm/ttypes.py
	storm-core/src/storm.thrift


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

Branch: refs/heads/master
Commit: 0571e22c15a1214c3e08510fddad608571f6a6d8
Parents: 6390064 8036109
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Mon Feb 9 12:47:36 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Mon Feb 9 12:47:36 2015 -0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   9 +
 README.markdown                                 |   8 +-
 STORM-UI-REST-API.md                            |  21 +
 dev-tools/github/__init__.py                    |  11 +
 dev-tools/jira-github-join.py                   |   4 +-
 dev-tools/storm-merge.py                        |  31 +
 docs/README.md                                  |   9 +
 docs/documentation/Common-patterns.md           |  14 +-
 docs/documentation/Concepts.md                  |  13 +-
 docs/documentation/Home.md                      |   2 +-
 docs/documentation/Multilang-protocol.md        |   4 +-
 docs/documentation/Powered-By.md                |   4 +-
 .../storm/starter/SkewedRollingTopWords.java    | 134 +++
 .../storm/starter/bolt/RollingCountAggBolt.java |  78 ++
 pom.xml                                         |  11 +
 storm-core/src/clj/backtype/storm/cluster.clj   |  43 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |   5 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  35 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |   6 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  34 +-
 .../coordination/BatchSubtopologyBuilder.java   |  11 +
 .../storm/drpc/LinearDRPCInputDeclarer.java     |   5 +-
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |  13 +-
 .../storm/generated/GetInfoOptions.java         | 350 +++++++
 .../jvm/backtype/storm/generated/Nimbus.java    | 974 +++++++++++++++++++
 .../storm/generated/NumErrorsChoice.java        |  64 ++
 .../storm/grouping/PartialKeyGrouping.java      |  31 +-
 .../backtype/storm/topology/InputDeclarer.java  |   3 +
 .../storm/topology/TopologyBuilder.java         |  11 +
 .../TransactionalTopologyBuilder.java           |  13 +-
 .../src/jvm/backtype/storm/utils/Monitor.java   |   8 +-
 .../topology/TridentTopologyBuilder.java        |  13 +-
 storm-core/src/py/storm/DistributedRPC-remote   |   0
 .../py/storm/DistributedRPCInvocations-remote   |   0
 storm-core/src/py/storm/Nimbus-remote           |   7 +
 storm-core/src/py/storm/Nimbus.py               | 226 +++++
 storm-core/src/py/storm/ttypes.py               |  80 ++
 storm-core/src/storm.thrift                     |  10 +
 .../clj/backtype/storm/integration_test.clj     |  10 +-
 .../storm/grouping/PartialKeyGroupingTest.java  |  26 +-
 40 files changed, 2272 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0571e22c/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/cluster.clj
index 15bf8a3,4b73f2e..1d5026f
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@@ -230,11 -239,11 +241,11 @@@
        (cb id))))
  
  (defn- maybe-deserialize
 -  [ser]
 +  [ser clazz]
    (when ser
 -    (Utils/deserialize ser)))
 +    (Utils/deserialize ser clazz)))
  
- (defstruct TaskError :error :time-secs :host :port)
+ (defrecord TaskError [error time-secs host port])
  
  (defn- parse-error-path
    [^String p]
@@@ -440,9 -441,13 +451,13 @@@
        (report-error
           [this storm-id component-id node port error]
           (let [path (error-path storm-id component-id)
+                last-error-path (last-error-path storm-id component-id)
 -               data {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port}
 +               data (thriftify-error {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port})
                 _ (mkdirs cluster-state path acls)
-                _ (create-sequential cluster-state (str path "/e") (Utils/serialize data) acls)
+                ser-data (Utils/serialize data)
+                _ (mkdirs cluster-state path acls)
+                _ (create-sequential cluster-state (str path "/e") ser-data acls)
+                _ (set-data cluster-state last-error-path ser-data acls)
                 to-kill (->> (get-children cluster-state path false)
                              (sort-by parse-error-path)
                              reverse
@@@ -455,16 -460,22 +470,24 @@@
           (let [path (error-path storm-id component-id)
                 errors (if (exists-node? cluster-state path false)
                          (dofor [c (get-children cluster-state path false)]
-                           (let [data (-> (get-data cluster-state (str path "/" c) false)
-                                        (maybe-deserialize ErrorInfo)
-                                        clojurify-error)]
-                             (when data
-                               (struct TaskError (:error data) (:time-secs data) (:host data) (:port data))
-                               )))
-                         ())
-                ]
+                           (if-let [data (-> (get-data cluster-state
+                                                       (str path "/" c)
+                                                       false)
 -                                         maybe-deserialize)]
++                                          (maybe-deserialize ErrorInfo)
++                                          clojurify-error)]
+                             (map->TaskError data)))
+                         ())]
             (->> (filter not-nil? errors)
                  (sort-by (comp - :time-secs)))))
+ 
+       (last-error
+         [this storm-id component-id]
+         (let [path (last-error-path storm-id component-id)]
+           (if (exists-node? cluster-state path false)
 -            (if-let [data (->> (get-data cluster-state path false)
 -                               maybe-deserialize)]
++            (if-let [data (-> (get-data cluster-state path false)
++                              (maybe-deserialize ErrorInfo)
++                              clojurify-error)]
+               (map->TaskError data)))))
        
        (disconnect
           [this]

http://git-wip-us.apache.org/repos/asf/storm/blob/0571e22c/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0571e22c/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0571e22c/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --cc storm-core/src/py/storm/ttypes.py
index e15cf1d,46e7a92..112daaa
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@@ -44,26 -44,23 +44,43 @@@ class TopologyInitialStatus
      "INACTIVE": 2,
    }
  
 +class TopologyStatus:
 +  ACTIVE = 1
 +  INACTIVE = 2
 +  REBALANCING = 3
 +  KILLED = 4
 +
 +  _VALUES_TO_NAMES = {
 +    1: "ACTIVE",
 +    2: "INACTIVE",
 +    3: "REBALANCING",
 +    4: "KILLED",
 +  }
 +
 +  _NAMES_TO_VALUES = {
 +    "ACTIVE": 1,
 +    "INACTIVE": 2,
 +    "REBALANCING": 3,
 +    "KILLED": 4,
 +  }
 +
+ class NumErrorsChoice:
+   ALL = 0
+   NONE = 1
+   ONE = 2
+ 
+   _VALUES_TO_NAMES = {
+     0: "ALL",
+     1: "NONE",
+     2: "ONE",
+   }
+ 
+   _NAMES_TO_VALUES = {
+     "ALL": 0,
+     "NONE": 1,
+     "ONE": 2,
+   }
+ 
  
  class JavaObjectArg:
    """
@@@ -4417,764 -4400,69 +4434,827 @@@ class SubmitOptions
    def __ne__(self, other):
      return not (self == other)
  
 +class SupervisorInfo:
 +  """
 +  Attributes:
 +   - time_secs
 +   - hostname
 +   - assignment_id
 +   - used_ports
 +   - meta
 +   - scheduler_meta
 +   - uptime_secs
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.I64, 'time_secs', None, None, ), # 1
 +    (2, TType.STRING, 'hostname', None, None, ), # 2
 +    (3, TType.STRING, 'assignment_id', None, None, ), # 3
 +    (4, TType.LIST, 'used_ports', (TType.I64,None), None, ), # 4
 +    (5, TType.LIST, 'meta', (TType.I64,None), None, ), # 5
 +    (6, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 6
 +    (7, TType.I64, 'uptime_secs', None, None, ), # 7
 +  )
 +
 +  def __hash__(self):
 +    return 0 + hash(self.time_secs) + hash(self.hostname) + hash(self.assignment_id) + hash(self.used_ports) + hash(self.meta) + hash(self.scheduler_meta) + hash(self.uptime_secs)
 +
 +  def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None,):
 +    self.time_secs = time_secs
 +    self.hostname = hostname
 +    self.assignment_id = assignment_id
 +    self.used_ports = used_ports
 +    self.meta = meta
 +    self.scheduler_meta = scheduler_meta
 +    self.uptime_secs = uptime_secs
 +
 +  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.I64:
 +          self.time_secs = iprot.readI64();
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.STRING:
 +          self.hostname = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 3:
 +        if ftype == TType.STRING:
 +          self.assignment_id = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 4:
 +        if ftype == TType.LIST:
 +          self.used_ports = []
 +          (_etype304, _size301) = iprot.readListBegin()
 +          for _i305 in xrange(_size301):
 +            _elem306 = iprot.readI64();
 +            self.used_ports.append(_elem306)
 +          iprot.readListEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 5:
 +        if ftype == TType.LIST:
 +          self.meta = []
 +          (_etype310, _size307) = iprot.readListBegin()
 +          for _i311 in xrange(_size307):
 +            _elem312 = iprot.readI64();
 +            self.meta.append(_elem312)
 +          iprot.readListEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 6:
 +        if ftype == TType.MAP:
 +          self.scheduler_meta = {}
 +          (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin() 
 +          for _i317 in xrange(_size313):
 +            _key318 = iprot.readString().decode('utf-8')
 +            _val319 = iprot.readString().decode('utf-8')
 +            self.scheduler_meta[_key318] = _val319
 +          iprot.readMapEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 7:
 +        if ftype == TType.I64:
 +          self.uptime_secs = iprot.readI64();
 +        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('SupervisorInfo')
 +    if self.time_secs is not None:
 +      oprot.writeFieldBegin('time_secs', TType.I64, 1)
 +      oprot.writeI64(self.time_secs)
 +      oprot.writeFieldEnd()
 +    if self.hostname is not None:
 +      oprot.writeFieldBegin('hostname', TType.STRING, 2)
 +      oprot.writeString(self.hostname.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    if self.assignment_id is not None:
 +      oprot.writeFieldBegin('assignment_id', TType.STRING, 3)
 +      oprot.writeString(self.assignment_id.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    if self.used_ports is not None:
 +      oprot.writeFieldBegin('used_ports', TType.LIST, 4)
 +      oprot.writeListBegin(TType.I64, len(self.used_ports))
 +      for iter320 in self.used_ports:
 +        oprot.writeI64(iter320)
 +      oprot.writeListEnd()
 +      oprot.writeFieldEnd()
 +    if self.meta is not None:
 +      oprot.writeFieldBegin('meta', TType.LIST, 5)
 +      oprot.writeListBegin(TType.I64, len(self.meta))
 +      for iter321 in self.meta:
 +        oprot.writeI64(iter321)
 +      oprot.writeListEnd()
 +      oprot.writeFieldEnd()
 +    if self.scheduler_meta is not None:
 +      oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6)
 +      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta))
 +      for kiter322,viter323 in self.scheduler_meta.items():
 +        oprot.writeString(kiter322.encode('utf-8'))
 +        oprot.writeString(viter323.encode('utf-8'))
 +      oprot.writeMapEnd()
 +      oprot.writeFieldEnd()
 +    if self.uptime_secs is not None:
 +      oprot.writeFieldBegin('uptime_secs', TType.I64, 7)
 +      oprot.writeI64(self.uptime_secs)
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    if self.time_secs is None:
 +      raise TProtocol.TProtocolException(message='Required field time_secs is unset!')
 +    if self.hostname is None:
 +      raise TProtocol.TProtocolException(message='Required field hostname is unset!')
 +    return
 +
 +
 +  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 NodeInfo:
 +  """
 +  Attributes:
 +   - node
 +   - port
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.STRING, 'node', None, None, ), # 1
 +    (2, TType.SET, 'port', (TType.I64,None), None, ), # 2
 +  )
 +
 +  def __hash__(self):
 +    return 0 + hash(self.node) + hash(self.port)
 +
 +  def __init__(self, node=None, port=None,):
 +    self.node = node
 +    self.port = port
 +
 +  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.STRING:
 +          self.node = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.SET:
 +          self.port = set()
 +          (_etype327, _size324) = iprot.readSetBegin()
 +          for _i328 in xrange(_size324):
 +            _elem329 = iprot.readI64();
 +            self.port.add(_elem329)
 +          iprot.readSetEnd()
 +        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('NodeInfo')
 +    if self.node is not None:
 +      oprot.writeFieldBegin('node', TType.STRING, 1)
 +      oprot.writeString(self.node.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    if self.port is not None:
 +      oprot.writeFieldBegin('port', TType.SET, 2)
 +      oprot.writeSetBegin(TType.I64, len(self.port))
 +      for iter330 in self.port:
 +        oprot.writeI64(iter330)
 +      oprot.writeSetEnd()
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    if self.node is None:
 +      raise TProtocol.TProtocolException(message='Required field node is unset!')
 +    if self.port is None:
 +      raise TProtocol.TProtocolException(message='Required field port is unset!')
 +    return
 +
 +
 +  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:
 +   - master_code_dir
 +   - node_host
 +   - executor_node_port
 +   - executor_start_time_secs
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.STRING, 'master_code_dir', None, None, ), # 1
 +    (2, TType.MAP, 'node_host', (TType.STRING,None,TType.STRING,None), {
 +    }, ), # 2
 +    (3, TType.MAP, 'executor_node_port', (TType.LIST,(TType.I64,None),TType.STRUCT,(NodeInfo, NodeInfo.thrift_spec)), {
 +    }, ), # 3
 +    (4, TType.MAP, 'executor_start_time_secs', (TType.LIST,(TType.I64,None),TType.I64,None), {
 +    }, ), # 4
 +  )
 +
 +  def __hash__(self):
 +    return 0 + hash(self.master_code_dir) + hash(self.node_host) + hash(self.executor_node_port) + hash(self.executor_start_time_secs)
 +
 +  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],):
 +    self.master_code_dir = master_code_dir
 +    if node_host is self.thrift_spec[2][4]:
 +      node_host = {
 +    }
 +    self.node_host = node_host
 +    if executor_node_port is self.thrift_spec[3][4]:
 +      executor_node_port = {
 +    }
 +    self.executor_node_port = executor_node_port
 +    if executor_start_time_secs is self.thrift_spec[4][4]:
 +      executor_start_time_secs = {
 +    }
 +    self.executor_start_time_secs = executor_start_time_secs
 +
 +  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.STRING:
 +          self.master_code_dir = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.MAP:
 +          self.node_host = {}
 +          (_ktype332, _vtype333, _size331 ) = iprot.readMapBegin() 
 +          for _i335 in xrange(_size331):
 +            _key336 = iprot.readString().decode('utf-8')
 +            _val337 = iprot.readString().decode('utf-8')
 +            self.node_host[_key336] = _val337
 +          iprot.readMapEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 3:
 +        if ftype == TType.MAP:
 +          self.executor_node_port = {}
 +          (_ktype339, _vtype340, _size338 ) = iprot.readMapBegin() 
 +          for _i342 in xrange(_size338):
 +            _key343 = []
 +            (_etype348, _size345) = iprot.readListBegin()
 +            for _i349 in xrange(_size345):
 +              _elem350 = iprot.readI64();
 +              _key343.append(_elem350)
 +            iprot.readListEnd()
 +            _val344 = NodeInfo()
 +            _val344.read(iprot)
 +            self.executor_node_port[_key343] = _val344
 +          iprot.readMapEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 4:
 +        if ftype == TType.MAP:
 +          self.executor_start_time_secs = {}
 +          (_ktype352, _vtype353, _size351 ) = iprot.readMapBegin() 
 +          for _i355 in xrange(_size351):
 +            _key356 = []
 +            (_etype361, _size358) = iprot.readListBegin()
 +            for _i362 in xrange(_size358):
 +              _elem363 = iprot.readI64();
 +              _key356.append(_elem363)
 +            iprot.readListEnd()
 +            _val357 = iprot.readI64();
 +            self.executor_start_time_secs[_key356] = _val357
 +          iprot.readMapEnd()
 +        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('Assignment')
 +    if self.master_code_dir is not None:
 +      oprot.writeFieldBegin('master_code_dir', TType.STRING, 1)
 +      oprot.writeString(self.master_code_dir.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    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 kiter364,viter365 in self.node_host.items():
 +        oprot.writeString(kiter364.encode('utf-8'))
 +        oprot.writeString(viter365.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 kiter366,viter367 in self.executor_node_port.items():
 +        oprot.writeListBegin(TType.I64, len(kiter366))
 +        for iter368 in kiter366:
 +          oprot.writeI64(iter368)
 +        oprot.writeListEnd()
 +        viter367.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 kiter369,viter370 in self.executor_start_time_secs.items():
 +        oprot.writeListBegin(TType.I64, len(kiter369))
 +        for iter371 in kiter369:
 +          oprot.writeI64(iter371)
 +        oprot.writeListEnd()
 +        oprot.writeI64(viter370)
 +      oprot.writeMapEnd()
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    if self.master_code_dir is None:
 +      raise TProtocol.TProtocolException(message='Required field master_code_dir is unset!')
 +    return
 +
 +
 +  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 TopologyActionOptions:
 +  """
 +  Attributes:
 +   - kill_options
 +   - rebalance_options
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.STRUCT, 'kill_options', (KillOptions, KillOptions.thrift_spec), None, ), # 1
 +    (2, TType.STRUCT, 'rebalance_options', (RebalanceOptions, RebalanceOptions.thrift_spec), None, ), # 2
 +  )
 +
 +  def __hash__(self):
 +    return 0 + hash(self.kill_options) + hash(self.rebalance_options)
 +
 +  def __init__(self, kill_options=None, rebalance_options=None,):
 +    self.kill_options = kill_options
 +    self.rebalance_options = rebalance_options
 +
 +  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.STRUCT:
 +          self.kill_options = KillOptions()
 +          self.kill_options.read(iprot)
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.STRUCT:
 +          self.rebalance_options = RebalanceOptions()
 +          self.rebalance_options.read(iprot)
 +        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('TopologyActionOptions')
 +    if self.kill_options is not None:
 +      oprot.writeFieldBegin('kill_options', TType.STRUCT, 1)
 +      self.kill_options.write(oprot)
 +      oprot.writeFieldEnd()
 +    if self.rebalance_options is not None:
 +      oprot.writeFieldBegin('rebalance_options', TType.STRUCT, 2)
 +      self.rebalance_options.write(oprot)
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    return
 +
 +
 +  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 StormBase:
 +  """
 +  Attributes:
 +   - name
 +   - status
 +   - num_workers
 +   - component_executors
 +   - launch_time_secs
 +   - owner
 +   - topology_action_options
 +   - prev_status
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.STRING, 'name', None, None, ), # 1
 +    (2, TType.I32, 'status', None, None, ), # 2
 +    (3, TType.I32, 'num_workers', None, None, ), # 3
 +    (4, TType.MAP, 'component_executors', (TType.STRING,None,TType.I32,None), None, ), # 4
 +    (5, TType.I32, 'launch_time_secs', None, None, ), # 5
 +    (6, TType.STRING, 'owner', None, None, ), # 6
 +    (7, TType.STRUCT, 'topology_action_options', (TopologyActionOptions, TopologyActionOptions.thrift_spec), None, ), # 7
 +    (8, TType.I32, 'prev_status', None, None, ), # 8
 +  )
 +
 +  def __hash__(self):
 +    return 0 + hash(self.name) + hash(self.status) + hash(self.num_workers) + hash(self.component_executors) + hash(self.launch_time_secs) + hash(self.owner) + hash(self.topology_action_options) + hash(self.prev_status)
 +
 +  def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None,):
 +    self.name = name
 +    self.status = status
 +    self.num_workers = num_workers
 +    self.component_executors = component_executors
 +    self.launch_time_secs = launch_time_secs
 +    self.owner = owner
 +    self.topology_action_options = topology_action_options
 +    self.prev_status = prev_status
 +
 +  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.STRING:
 +          self.name = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.I32:
 +          self.status = iprot.readI32();
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 3:
 +        if ftype == TType.I32:
 +          self.num_workers = iprot.readI32();
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 4:
 +        if ftype == TType.MAP:
 +          self.component_executors = {}
 +          (_ktype373, _vtype374, _size372 ) = iprot.readMapBegin() 
 +          for _i376 in xrange(_size372):
 +            _key377 = iprot.readString().decode('utf-8')
 +            _val378 = iprot.readI32();
 +            self.component_executors[_key377] = _val378
 +          iprot.readMapEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 5:
 +        if ftype == TType.I32:
 +          self.launch_time_secs = iprot.readI32();
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 6:
 +        if ftype == TType.STRING:
 +          self.owner = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 7:
 +        if ftype == TType.STRUCT:
 +          self.topology_action_options = TopologyActionOptions()
 +          self.topology_action_options.read(iprot)
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 8:
 +        if ftype == TType.I32:
 +          self.prev_status = iprot.readI32();
 +        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('StormBase')
 +    if self.name is not None:
 +      oprot.writeFieldBegin('name', TType.STRING, 1)
 +      oprot.writeString(self.name.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    if self.status is not None:
 +      oprot.writeFieldBegin('status', TType.I32, 2)
 +      oprot.writeI32(self.status)
 +      oprot.writeFieldEnd()
 +    if self.num_workers is not None:
 +      oprot.writeFieldBegin('num_workers', TType.I32, 3)
 +      oprot.writeI32(self.num_workers)
 +      oprot.writeFieldEnd()
 +    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 kiter379,viter380 in self.component_executors.items():
 +        oprot.writeString(kiter379.encode('utf-8'))
 +        oprot.writeI32(viter380)
 +      oprot.writeMapEnd()
 +      oprot.writeFieldEnd()
 +    if self.launch_time_secs is not None:
 +      oprot.writeFieldBegin('launch_time_secs', TType.I32, 5)
 +      oprot.writeI32(self.launch_time_secs)
 +      oprot.writeFieldEnd()
 +    if self.owner is not None:
 +      oprot.writeFieldBegin('owner', TType.STRING, 6)
 +      oprot.writeString(self.owner.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    if self.topology_action_options is not None:
 +      oprot.writeFieldBegin('topology_action_options', TType.STRUCT, 7)
 +      self.topology_action_options.write(oprot)
 +      oprot.writeFieldEnd()
 +    if self.prev_status is not None:
 +      oprot.writeFieldBegin('prev_status', TType.I32, 8)
 +      oprot.writeI32(self.prev_status)
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    if self.name is None:
 +      raise TProtocol.TProtocolException(message='Required field name is unset!')
 +    if self.status is None:
 +      raise TProtocol.TProtocolException(message='Required field status is unset!')
 +    if self.num_workers is None:
 +      raise TProtocol.TProtocolException(message='Required field num_workers is unset!')
 +    return
 +
 +
 +  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 ZKWorkerHeartbeat:
 +  """
 +  Attributes:
 +   - storm_id
 +   - executor_stats
 +   - time_secs
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.STRING, 'storm_id', None, None, ), # 1
 +    (2, TType.MAP, 'executor_stats', (TType.STRUCT,(ExecutorInfo, ExecutorInfo.thrift_spec),TType.STRUCT,(ExecutorStats, ExecutorStats.thrift_spec)), None, ), # 2
 +    (3, TType.I32, 'time_secs', None, None, ), # 3
 +  )
 +
 +  def __hash__(self):
 +    return 0 + hash(self.storm_id) + hash(self.executor_stats) + hash(self.time_secs)
 +
 +  def __init__(self, storm_id=None, executor_stats=None, time_secs=None,):
 +    self.storm_id = storm_id
 +    self.executor_stats = executor_stats
 +    self.time_secs = time_secs
 +
 +  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.STRING:
 +          self.storm_id = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.MAP:
 +          self.executor_stats = {}
 +          (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin() 
 +          for _i385 in xrange(_size381):
 +            _key386 = ExecutorInfo()
 +            _key386.read(iprot)
 +            _val387 = ExecutorStats()
 +            _val387.read(iprot)
 +            self.executor_stats[_key386] = _val387
 +          iprot.readMapEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 3:
 +        if ftype == TType.I32:
 +          self.time_secs = iprot.readI32();
 +        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('ZKWorkerHeartbeat')
 +    if self.storm_id is not None:
 +      oprot.writeFieldBegin('storm_id', TType.STRING, 1)
 +      oprot.writeString(self.storm_id.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    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 kiter388,viter389 in self.executor_stats.items():
 +        kiter388.write(oprot)
 +        viter389.write(oprot)
 +      oprot.writeMapEnd()
 +      oprot.writeFieldEnd()
 +    if self.time_secs is not None:
 +      oprot.writeFieldBegin('time_secs', TType.I32, 3)
 +      oprot.writeI32(self.time_secs)
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    if self.storm_id is None:
 +      raise TProtocol.TProtocolException(message='Required field storm_id is unset!')
 +    if self.executor_stats is None:
 +      raise TProtocol.TProtocolException(message='Required field executor_stats is unset!')
 +    if self.time_secs is None:
 +      raise TProtocol.TProtocolException(message='Required field time_secs is unset!')
 +    return
 +
 +
 +  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 GetInfoOptions:
+   """
+   Attributes:
+    - num_err_choice
+   """
+ 
+   thrift_spec = (
+     None, # 0
+     (1, TType.I32, 'num_err_choice', None, None, ), # 1
+   )
+ 
+   def __hash__(self):
+     return 0 + hash(self.num_err_choice)
+ 
+   def __init__(self, num_err_choice=None,):
+     self.num_err_choice = num_err_choice
+ 
+   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.I32:
+           self.num_err_choice = iprot.readI32();
+         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('GetInfoOptions')
+     if self.num_err_choice is not None:
+       oprot.writeFieldBegin('num_err_choice', TType.I32, 1)
+       oprot.writeI32(self.num_err_choice)
+       oprot.writeFieldEnd()
+     oprot.writeFieldStop()
+     oprot.writeStructEnd()
+ 
+   def validate(self):
+     return
+ 
+ 
+   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 DRPCRequest:
    """
    Attributes:

http://git-wip-us.apache.org/repos/asf/storm/blob/0571e22c/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --cc storm-core/src/storm.thrift
index 3cc0eb9,066cb4f..04b6a1b
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@@ -244,55 -243,15 +244,64 @@@ struct SubmitOptions 
    2: optional Credentials creds;
  }
  
 +struct SupervisorInfo {
 +    1: required i64 time_secs;
 +    2: required string hostname;
 +    3: optional string assignment_id;
 +    4: optional list<i64> used_ports;
 +    5: optional list<i64> meta;
 +    6: optional map<string, string> scheduler_meta;
 +    7: optional i64 uptime_secs;
 +}
 +struct NodeInfo {
 +    1: required string node;
 +    2: required set<i64> port;
 +}
 +
 +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 = {};
 +}
 +
 +enum TopologyStatus {
 +    ACTIVE = 1,
 +    INACTIVE = 2,
 +    REBALANCING = 3,
 +    KILLED = 4
 +}
 +
 +union TopologyActionOptions {
 +    1: optional KillOptions kill_options;
 +    2: optional RebalanceOptions rebalance_options;
 +}
 +
 +struct StormBase {
 +    1: required string name;
 +    2: required TopologyStatus status;
 +    3: required i32 num_workers;
 +    4: optional map<string, i32> component_executors;
 +    5: optional i32 launch_time_secs;
 +    6: optional string owner;
 +    7: optional TopologyActionOptions topology_action_options;
 +    8: optional TopologyStatus prev_status;//currently only used during rebalance action.
 +}
 +
 +struct ZKWorkerHeartbeat {
 +    1: required string storm_id;
 +    2: required map<ExecutorInfo,ExecutorStats> executor_stats;
 +    3: required i32 time_secs;
 +}
+ enum NumErrorsChoice {
+   ALL,
+   NONE,
+   ONE
+ }
+ 
+ struct GetInfoOptions {
+   1: optional NumErrorsChoice num_err_choice;
+ }
  
  service Nimbus {
    void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);