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);