Full Code of nathanmarz/storm for AI

moved-to-apache cdb116e94266 cached
585 files
3.2 MB
885.6k tokens
7237 symbols
1 requests
Download .txt
Showing preview only (3,530K chars total). Download the full file or copy to clipboard to get everything.
Repository: nathanmarz/storm
Branch: moved-to-apache
Commit: cdb116e94266
Files: 585
Total size: 3.2 MB

Directory structure:
gitextract_bzfpz0c3/

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

================================================
FILE CONTENTS
================================================

================================================
FILE: .gitignore
================================================
.lein-repl-history
/classes
/lib
deploy/lib
deploy/logs
.emacs-project
*.jar
bin/jzmq
.DS_Store
pom.xml
deploy/classes
*.fyc
*.rbc
*.pyc
CHILD
CHILDMAKER
NANNY
\#project.clj\#
.\#project.clj
.lein-failures
_release
*.zip
*.tar.gz
.lein-deps-sum
*.iml
target
/.project/
/.lein-plugins/
*.ipr
*.iws
.idea


================================================
FILE: CHANGELOG.md
================================================
## 0.9.0.1
* Update build configuration to force compatibility with Java 1.6

## 0.9.0
* Fixed a netty client issue where sleep times for reconnection could be negative (thanks brndnmtthws)
* Fixed an issue that would cause storm-netty unit tests to fail

## 0.9.0-rc3
* Added configuration to limit ShellBolt internal _pendingWrites queue length (thanks xiaokang)
* Fixed a a netty client issue where sleep times for reconnection could be negative (thanks brndnmtthws)
* Fixed a display issue with system stats in Storm UI (thanks d2r)
* Nimbus now does worker heartbeat timeout checks as soon as heartbeats are updated (thanks d2r)
* The logviewer now determines log file location by examining the logback configuration (thanks strongh)
* Allow tick tuples to work with the system bolt (thanks xumingming)
* Add default configuration values for the netty transport and the ability to configure the number of worker threads (thanks revans2)
* Added timeout to unit tests to prevent a situation where tests would hang indefinitely (thanks d2r)
* Fixed and issue in the system bolt where local mode would not be detected accurately (thanks miofthena)

## 0.9.0-rc2 

* Fixed `storm jar` command to work properly when STORM_JAR_JVM_OPTS is not specified (thanks roadkill001)

## 0.9.0-rc1

 * All logging now done with slf4j
 * Replaced log4j logging system with logback
 * Logs are now limited to 1GB per worker (configurable via logging configuration file)
 * Build upgraded to leiningen 2.0
 * Revamped Trident spout interfaces to support more dynamic spouts, such as a spout who reads from a changing set of brokers
 * How tuples are serialized is now pluggable (thanks anfeng)
 * Added blowfish encryption based tuple serialization (thanks anfeng)
 * Have storm fall back to installed storm.yaml (thanks revans2)
 * Improve error message when Storm detects bundled storm.yaml to show the URL's for offending resources (thanks revans2)
 * Nimbus throws NotAliveException instead of FileNotFoundException from various query methods when topology is no longer alive (thanks revans2)
 * Escape HTML and Javascript appropriately in Storm UI (thanks d2r)
 * Storm's Zookeeper client now uses bounded exponential backoff strategy on failures
 * Automatically drain and log error stream of multilang subprocesses
 * Append component name to thread name of running executors so that logs are easier to read
 * Messaging system used for passing messages between workers is now pluggable (thanks anfeng)
 * Netty implementation of messaging (thanks anfeng)
 * Include topology id, worker port, and worker id in properties for worker processes, useful for logging (thanks d2r)
 * Tick tuples can now be scheduled using floating point seconds (thanks tscurtu)
 * Added log viewer daemon and links from UI to logviewers (thanks xiaokang)
 * DRPC server childopts now configurable (thanks strongh)
 * Default number of ackers to number of workers, instead of just one (thanks lyogavin)
 * Validate that Storm configs are of proper types/format/structure (thanks d2r)
 * FixedBatchSpout will now replay batches appropriately on batch failure (thanks ptgoetz)
 * Can set JAR_JVM_OPTS env variable to add jvm options when calling 'storm jar' (thanks srmelody)
 * Throw error if batch id for transaction is behind the batch id in the opaque value (thanks mrflip)
 * Sort topologies by name in UI (thanks jaked)
 * Added LoggingMetricsConsumer to log all metrics to a file, by default not enabled (thanks mrflip)
 * Add prepare(Map conf) method to TopologyValidator (thanks ankitoshniwal)
 * Bug fix: Supervisor provides full path to workers to logging config rather than relative path (thanks revans2) 
 * Bug fix: Call ReducerAggregator#init properly when used within persistentAggregate (thanks lorcan)
 * Bug fix: Set component-specific configs correctly for Trident spouts

## 0.8.3 (unreleased)

 * Revert zmq layer to not rely on multipart messages to fix issue reported by some users
 * Bug fix: Fix TransactionalMap and OpaqueMap to correctly do multiple updates to the same key in the same batch
 * Bug fix: Fix race condition between supervisor and Nimbus that could lead to stormconf.ser errors and infinite crashing of supervisor
 * Bug fix: Fix default scheduler to always reassign workers in a constrained topology when there are dead executors
 * Bug fix: Fix memory leak in Trident LRUMemoryMapState due to concurrency issue with LRUMap (thanks jasonjckn)
 * Bug fix: Properly ignore NoNodeExists exceptions when deleting old transaction states

## 0.8.2

 * Added backtype.storm.scheduler.IsolationScheduler. This lets you run topologies that are completely isolated at the machine level. Configure Nimbus to isolate certain topologies, and how many machines to give to each of those topologies, with the isolation.scheduler.machines config in Nimbus's storm.yaml. Topologies run on the cluster that are not listed there will share whatever remaining machines there are on the cluster after machines are allocated to the listed topologies.
 * Storm UI now uses nimbus.host to find Nimbus rather than always using localhost (thanks Frostman)
 * Added report-error! to Clojure DSL
 * Automatically throttle errors sent to Zookeeper/Storm UI when too many are reported in a time interval (all errors are still logged) Configured with TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL and TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS
 * Kryo instance used for serialization can now be controlled via IKryoFactory interface and TOPOLOGY_KRYO_FACTORY config
 * Add ability to plug in custom code into Nimbus to allow/disallow topologies to be submitted via NIMBUS_TOPOLOGY_VALIDATOR config
 * Added TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS config to control how often a batch can be emitted in a Trident topology. Defaults to 500 milliseconds. This is used to prevent too much load from being placed on Zookeeper in the case that batches are being processed super quickly.
 * Log any topology submissions errors in nimbus.log
 * Add static helpers in Config when using regular maps
 * Make Trident much more memory efficient during failures by immediately removing state for failed attempts when a more recent attempt is seen
 * Add ability to name portions of a Trident computation and have those names appear in the Storm UI
 * Show Nimbus and topology configurations through Storm UI (thanks rnfein)
 * Added ITupleCollection interface for TridentState's and TupleCollectionGet QueryFunction for getting the full contents of a state. MemoryMapState and LRUMemoryMapState implement this
 * Can now submit a topology in inactive state. Storm will wait to call open/prepare on the spouts/bolts until it is first activated.
 * Can now activate, deactive, rebalance, and kill topologies from the Storm UI (thanks Frostman)
 * Can now use --config option to override which yaml file from ~/.storm to use for the config (thanks tjun)
 * Redesigned the pluggable resource scheduler (INimbus, ISupervisor) interfaces to allow for much simpler integrations
 * Added prepare method to IScheduler
 * Added "throws Exception" to TestJob interface
 * Added reportError to multilang protocol and updated Python and Ruby adapters to use it (thanks Lazyshot)
 * Number tuples executed now tracked and shown in Storm UI
 * Added ReportedFailedException which causes a batch to fail without killing worker and reports the error to the UI
 * Execute latency now tracked and shown in Storm UI
 * Adding testTuple methods for easily creating Tuple instances to Testing API (thanks xumingming)
 * Trident now throws an error during construction of a topology when try to select fields that don't exist in a stream (thanks xumingming)
 * Compute the capacity of a bolt based on execute latency and #executed over last 10 minutes and display in UI
 * Storm UI displays exception instead of blank page when there's an error rendering the page (thanks Frostman)
 * Added MultiScheme interface (thanks sritchie)
 * Added MockTridentTuple for testing (thanks emblem)
 * Add whitelist methods to Cluster to allow only a subset of hosts to be revealed as available slots
 * Updated Trident Debug filter to take in an identifier to use when logging (thanks emblem)
 * Number of DRPC server worker threads now customizable (thanks xiaokang)
 * DRPC server now uses a bounded queue for requests to prevent being overloaded with requests (thanks xiaokang)
 * Add __hash__ method to all generated Python Thrift objects so that Python code can read Nimbus stats which use Thrift objects as dict keys
 * Bug fix: Fix for bug that could cause topology to hang when ZMQ blocks sending to a worker that got reassigned
 * Bug fix: Fix deadlock bug due to variant of dining philosophers problem. Spouts now use an overflow buffer to prevent blocking and guarantee that it can consume the incoming queue of acks/fails.
 * Bug fix: Fix race condition in supervisor that would lead to supervisor continuously crashing due to not finding "stormconf.ser" file for an already killed topology
 * Bug fix: bin/storm script now displays a helpful error message when an invalid command is specified
 * Bug fix: fixed NPE when emitting during emit method of Aggregator
 * Bug fix: URLs with periods in them in Storm UI now route correctly
 * Bug fix: Fix occasional cascading worker crashes due when a worker dies due to not removing connections from connection cache appropriately
  
## 0.8.1

 * Exposed Storm's unit testing facilities via the backtype.storm.Testing class. Notable functions are Testing/withLocalCluster and Testing/completeTopology (thanks xumingming)
 * Implemented pluggable spout wait strategy that is invoked when a spout emits nothing from nextTuple or when a spout hits the MAX_SPOUT_PENDING limit
 * Spouts now have a default wait strategy of a 1 millisecond sleep
 * Changed log level of "Failed message" logging to DEBUG
 * Deprecated LinearDRPCTopologyBuilder, TimeCacheMap, and transactional topologies
 * During "storm jar", whether topology is already running or not is checked before submitting jar to save time (thanks jasonjckn)
 * Added BaseMultiReducer class to Trident that provides empty implementations of prepare and cleanup
 * Added Negate builtin operation to reverse a Filter
 * Added topology.kryo.decorators config that allows functions to be plugged in to customize Kryo (thanks jasonjckn)
 * Enable message timeouts when using LocalCluster
 * Multilang subprocesses can set "need_task_ids" to false when emitting tuples to tell Storm not to send task ids back (performance optimization) (thanks barrywhart)
 * Add contains method on Tuple (thanks okapies)
 * Added ISchemableSpout interface
 * Bug fix: When an item is consumed off an internal buffer, the entry on the buffer is nulled to allow GC to happen on that data
 * Bug fix: Helper class for Trident MapStates now clear their read cache when a new commit happens, preventing updates from spilling over from a failed batch attempt to the next attempt
 * Bug fix: Fix NonTransactionalMap to take in an IBackingMap for regular values rather than TransactionalValue (thanks sjoerdmulder)
 * Bug fix: Fix NPE when no input fields given for regular Aggregator
 * Bug fix: Fix IndexOutOfBoundsExceptions when a bolt for global aggregation had a parallelism greater than 1 (possible with splitting, stateQuerying, and multiReduce)
 * Bug fix: Fix "fields size" error that would sometimes occur when splitting a stream with multiple eaches
 * Bug fix: Fix bug where a committer spout (including opaque spouts) could cause Trident batches to fail
 * Bug fix: Fix Trident bug where multiple groupings on same stream would cause tuples to be duplicated to all consumers
 * Bug fix: Fixed error when repartitioning stream twice in a row without any operations in between
 * Bug fix: Fix rare bug in supervisor where it would continuously fail to clean up workers because the worker was already partially cleaned up
 * Bug fix: Fix emitDirect in storm.py

## 0.8.0

 * Added Trident, the new high-level abstraction for intermixing high throughput, stateful stream processing with low-latency distributed querying
 * Added executor abstraction between workers and tasks. Workers = processes, executors = threads that run many tasks from the same spout or bolt.
 * Pluggable scheduler (thanks xumingming)
 * Eliminate explicit storage of task->component in Zookeeper
 * Number of workers can be dynamically changed at runtime through rebalance command and -n switch
 * Number of executors for a component can be dynamically changed at runtime through rebalance command and -e switch (multiple -e switches allowed)
 * Use worker heartbeats instead of task heartbeats (thanks xumingming)
 * UI performance for topologies with many executors/tasks much faster due to optimized usage of Zookeeper (10x improvement)
 * Added button to show/hide system stats (e.g., acker component and stream stats) from the Storm UI (thanks xumingming)
 * Stats are tracked on a per-executor basis instead of per-task basis
 * Major optimization for unreliable spouts and unanchored tuples (will use far less CPU)
 * Revamped internals of Storm to use LMAX disruptor for internal queuing. Dramatic reductions in contention and CPU usage.
 * Numerous micro-optimizations all throughout the codebase to reduce CPU usage.
 * Optimized internals of Storm to use much fewer threads - two fewer threads per spout and one fewer thread per acker.
 * Removed error method from task hooks (to be re-added at a later time)
 * Validate that subscriptions come from valid components and streams, and if it's a field grouping that the schema is correct (thanks xumingming)
 * MemoryTransactionalSpout now works in cluster mode
 * Only track errors on a component by component basis to reduce the amount stored in zookeeper (to speed up UI). A side effect of this change is the removal of the task page in the UI.
 * Add TOPOLOGY-TICK-TUPLE-FREQ-SECS config to have Storm automatically send "tick" tuples to a bolt's execute method coming from the __system component and __tick stream at the configured frequency. Meant to be used as a component-specific configuration.
 * Upgrade Kryo to v2.17
 * Tuple is now an interface and is much cleaner. The Clojure DSL helpers have been moved to TupleImpl
 * Added shared worker resources. Storm provides a shared ExecutorService thread pool by default. The number of threads in the pool can be configured with topology.worker.shared.thread.pool.size
 * Improve CustomStreamGrouping interface to make it more flexible by providing more information
 * Enhanced INimbus interface to allow for forced schedulers and better integration with global scheduler
 * Added assigned method to ISupervisor so it knows exactly what's running and not running
 * Custom serializers can now have one of four constructors: (), (Kryo), (Class), or (Kryo, Class)
 * Disallow ":", ".", and "\" from topology names
 * Errors in multilang subprocesses that go to stderr will be captured and logged to the worker logs (thanks vinodc)
 * Workers detect and warn for missing outbound connections from assignment, drop messages for which there's no outbound connection
 * Zookeeper connection timeout is now configurable (via storm.zookeeper.connection.timeout config)
 * Storm is now less aggressive about halting process when there are Zookeeper errors, preferring to wait until client calls return exceptions.
 * Can configure Zookeeper authentication for Storm's Zookeeper clients via "storm.zookeeper.auth.scheme" and "storm.zookeeper.auth.payload" configs
 * Supervisors only download code for topologies assigned to them
 * Include task id information in task hooks (thanks velvia)
 * Use execvp to spawn daemons (replaces the python launcher process) (thanks ept)
 * Expanded INimbus/ISupervisor interfaces to provide more information (used in Storm/Mesos integration)
 * Bug fix: Realize task ids when worker heartbeats to supervisor. Some users were hitting deserialization problems here in very rare cases (thanks herberteuler)
 * Bug fix: Fix bug where a topology's status would get corrupted to true if nimbus is restarted while status is rebalancing

## 0.7.4

 * Bug fix: Disallow slashes in topology names since it causes Nimbus to break by affecting local filesystem and zookeeper paths
 * Bug fix: Prevent slow loading tasks from causing worker timeouts by launching the heartbeat thread before tasks are loaded

## 0.7.3

 * Changed debug level of "Failed message" logging to DEBUG
 * Bug fix: Fixed critical regression in 0.7.2 that could cause workers to timeout to the supervisors or to Nimbus. 0.7.2 moved all system tasks to the same thread, so if one took a long time it would block the other critical tasks. Now different system tasks run on different threads.

## 0.7.2

NOTE: The change from 0.7.0 in which OutputCollector no longer assumes immutable inputs has been reverted to support optimized sending of tuples to colocated tasks

 * Messages sent to colocated tasks are sent in-memory, skipping serialization (useful in conjunction with localOrShuffle grouping) (thanks xumingming)
 * Upgrade to Clojure 1.4 (thanks sorenmacbeth)
 * Exposed INimbus and ISupervisor interfaces for running Storm on different resource frameworks (like Mesos).
 * Can override the hostname that supervisors report using "storm.local.hostname" config.
 * Make request timeout within DRPC server configurable via "drpc.request.timeout.secs"
 * Added "storm list" command to show running topologies at the command line (thanks xumingming)
 * Storm UI displays the release version (thanks xumingming)
 * Added reportError to BasicOutputCollector
 * Added reportError to BatchOutputCollector
 * Added close method to OpaqueTransactionalSpout coordinator
 * Added "storm dev-zookeeper" command for launching a local zookeeper server. Useful for testing a one node Storm cluster locally. Zookeeper dir configured with "dev.zookeeper.path"
 * Use new style classes for Python multilang adapter (thanks hellp)
 * Added "storm version" command
 * Heavily refactored and simplified the supervisor and worker code
 * Improved error message when duplicate config files found on classpath
 * Print the host and port of Nimbus when using the storm command line client
 * Include as much of currently read output as possible when pipe to subprocess is broken in multilang components
 * Lower supervisor worker start timeout to 120 seconds
 * More debug logging in supervisor
 * "nohup" no longer used by supervisor to launch workers (unnecessary)
 * Throw helpful error message if StormSubmitter used without using storm client script
 * Add Values class as a default serialization
 * Bug fix: give absolute piddir to subprocesses (so that relative paths can be used for storm local dir)
 * Bug fix: Fixed critical bug in transactional topologies where a batch would be considered successful even if the batch didn't finish
 * Bug fix: Fixed critical bug in opaque transactional topologies that would lead to duplicate messages when using pipelining
 * Bug fix: Workers will now die properly if a ShellBolt subprocess dies (thanks tomo)
 * Bug fix: Hide the BasicOutputCollector#getOutputter method, since it shouldn't be a publicly available method
 * Bug fix: Zookeeper in local mode now always gets an unused port. This will eliminate conflicts with other local mode processes or other Zookeeper instances on a local machine. (thanks xumingming)
 * Bug fix: Fixed NPE in CoordinatedBolt it tuples emitted, acked, or failed for a request id that has already timed out. (thanks xumingming)
 * Bug fix: UI no longer errors for topologies with no assigned tasks (thanks xumingming)
 * Bug fix: emitDirect on SpoutOutputCollector now works
 * Bug fix: Fixed NPE when giving null parallelism hint for spout in TransactionalTopologyBuilder (thanks xumingming)

## 0.7.1

 * Implemented shell spout (thanks tomo)
 * Shell bolts can now asynchronously emit/ack messages (thanks tomo)
 * Added hooks for when a tuple is emitted, acked, or failed in bolts or spouts.
 * Added activate and deactivate lifecycle methods on spouts. Spouts start off deactivated.
 * Added isReady method to ITransactionalSpout$Coordinator to give the ability to delay the creation of new batches
 * Generalized CustomStreamGrouping to return the target tasks rather than the indices. Also parameterized custom groupings with TopologyContext. (not backwards compatible)
 * Added localOrShuffle grouping that will send to tasks in the same worker process if possible, or do a shuffle grouping otherwise.
 * Removed parameter from TopologyContext#maxTopologyMessageTimeout (simplification).
 * Storm now automatically sets TOPOLOGY_NAME in the config passed to the bolts and spouts to the name of the topology.
 * Added TOPOLOGY_AUTO_TASK_HOOKS config to automatically add hooks into every spout/bolt for the topology.
 * Added ability to override configs at the command line. These config definitions have the highest priority.
 * Error thrown if invalid (not json-serializable) topology conf used.
 * bin/storm script can now be symlinked (thanks gabrielgrant)
 * Socket timeout for DRPCClient is now configurable
 * Added getThisWorkerPort() method to TopologyContext
 * Added better error checking in Fields (thanks git2samus)
 * Improved Clojure DSL to allow destructuring in bolt/spout methods
 * Added Nimbus stats methods to LocalCluster (thanks KasperMadsen)
 * Added rebalance, activate, deactivate, and killTopologyWithOpts methods to LocalCluster
 * Added custom stream groupings to LinearDRPC API
 * Simplify multilang protocol to use json for all messages (thanks tomoj)
 * Bug fix: Fixed string encoding in ShellBolt protocol to be UTF-8 (thanks nicoo)
 * Bug fix: Fixed race condition in FeederSpout that could lead to dropped messages
 * Bug fix: Quoted arguments with spaces now work properly with storm client script
 * Bug fix: Workers start properly when topology name has spaces
 * Bug fix: UI works properly when there are spaces in topology or spout/bolt names (thanks xiaokang)
 * Bug fix: Tuple$Seq now returns correct count (thanks travisfw)

## 0.7.0

 * Transactional topologies: a new higher level abstraction that enables exactly-once messaging semantics for most computations. Documented on the wiki.
 * Component-specific configurations: Can now set configurations on a per-spout or per-bolt basis. 
 * New batch bolt abstraction that simplifies the processing of batches in DRPC or transactional topologies. A new batch bolt is created per batch and they are automatically cleaned up.
 * Introduction of base classes for various bolt and spout types. These base classes are in the backtype.storm.topology.base package and provide empty implementations for commonly unused methods
 * CoordinatedBolt generalized to handle non-linear topologies. This will make it easy to implement a non-linear DRPC topology abstraction.
 * Can customize the JVM options for Storm UI with new ui.childopts config
 * BigIntegers are now serializable by default
 * All bolts/spouts now emit a system stream (id "__system"). Currently it only emits startup events, but may emit other events in the future.
 * Optimized tuple trees for batch processing in DRPC and transactional topologies. Only the coordination tuples are anchored. OutputCollector#fail still works because CoordinatedBolt will propagate the fail to all other tuples in the batch. 
 * CoordinatedBolt moved to backtype.storm.coordination package
 * Clojure test framework significantly more composable
 * Massive internal refactorings and simplifications, including changes to the Thrift definition for storm topologies.
 * Optimized acking system. Bolts with zero or more than one consumer used to send an additional ack message. Now those are no longer sent.
 * Changed interface of CustomStreamGrouping to receive a List<Object> rather than a Tuple.
 * Added "storm.zookeeper.retry.times" and "storm.zookeeper.retry.interval" configs (thanks killme2008)
 * Added "storm help" and "storm help {cmd}" to storm script (thanks kachayev)
 * Logging now always goes to logs/ in the Storm directory, regardless of where you launched the daemon (thanks haitaoyao)
 * Improved Clojure DSL: can emit maps and Tuples implement the appropriate interfaces to integrate with Clojure's seq functions (thanks schleyfox)
 * Added "ui.childopts" config (thanks ddillinger)
 * Bug fix: OutputCollector no longer assumes immutable inputs [NOTE: this was reverted in 0.7.2 because it conflicts with sending tuples to colocated tasks without serialization]
 * Bug fix: DRPC topologies now throw a proper error when no DRPC servers are configured instead of NPE (thanks danharvey)
 * Bug fix: Fix local mode so multiple topologies can be run on one LocalCluster
 * Bug fix: "storm supervisor" now uses supervisor.childopts instead of nimbus.childopts (thanks ddillinger)
 * Bug fix: supervisor.childopts and nimbus.childopts can now contain whitespace. Previously only the first token was taken from the string
 * Bug fix: Make TopologyContext "getThisTaskIndex" and "getComponentTasks" consistent
 * Bug fix: Fix NoNodeException that would pop up with task heartbeating under heavy load
 * Bug fix: Catch InterruptedExceptions appropriately in local mode so shutdown always works properly

## 0.6.2

 * Automatically delete old files in Nimbus's inbox. Configurable with "nimbus.cleanup.inbox.freq.secs" and "nimbus.inbox.jar.expiration.secs"
 * Redirect System.out and System.err to log4j
 * Added "topology.worker.child.opts" config, for topology-configurable worker options.
 * Use Netflix's Curator library for Zookeeper communication. Workers now reconnect to Zookeeper rather than crash when there's a disconnection.
 * Bug fix: DRPC server no longer hangs with too many concurrent requests. DPRC server now requires two ports: "drpc.port" and "drpc.invocations.port"
 * Bug fix: Multilang resources are now extracted from the relevant jar on the classpath when appropriate. Previously an error would be thrown if the resources/ dir was in a jar in local mode.
 * Bug fix: Fix race condition in unit testing where time simulation fails to detect that Storm cluster is waiting due to threads that are not alive
 * Bug fix: Fix deadlock in Nimbus that could be triggered by a kill command.

## 0.6.1

 * storm client "activate" and "deactivate" commands
 * storm client "rebalance" command
 * Nimbus will automatically detect and cleanup corrupt topologies (this would previously give an error of the form "file storm...ser cannot be found").
 * "storm" client will not run unless it's being used from a release. 
 * Topology jar path now passed in using a java property rather than an environment variable.
 * LD\_LIBRARY\_PATH environment variable is now set on worker processes appropriately.
 * Replaced jvyaml with snakeyaml. UTF-8 YAML files should now work properly. 
 * Upgraded httpclient, httpcore, and commons-codec dependencies.

## 0.6.0

 * New serialization system based on Kryo
 * Component and stream ids are now strings
 * Pluggable stream groupings
 * Storm now chooses an unused port for Zookeeper in local mode instead of crashing when 2181 was in use.
 * Better support for defining topologies in non-JVM languages. The Thrift structure for topologies now allows you to specify components using a Java class name and a list of arguments to that class's constructor.
 * Bug fix: errors during the preparation phase of spouts or bolts will be reported to the Storm UI 
 * Bug fix: Fixed bugs related to LinearDRPC topologies where the last bolt implements FinishedCallback 
 * Bug fix: String greater than 64K will now serialize properly 
 * Generalized type of anchors in OutputCollector methods to Collection from List. 
 * Improved logging throughout.
 * In the "worker.childopts" config, %ID% will be replaced by the worker port. 
 * Significant internal refactorings to clean up the codebase. 

## 0.5.4

 * LinearDRPCTopologyBuilder, a polished DRPC implementation, 
 * Improved custom serialization support. no longer need to provide "token" ids. 
 * Fallback on Java serialization by default. Can be turned off by setting "topology.fall.back.on.java.serialization" to false. 
 * Improved "storm kill" command. Can override the wait time with "-w" flag.
 * Display topology status in Storm UI
 * Changed Thrift namespace to avoid conflicts
 * Better error messages throughout
 * Storm UI port is configurable through "ui.port" 
 * Minor improvements to Clojure DSL 

## 0.5.3

 * Nimbus and supervisor daemons can now share a local dir. 
 * Greatly improved Clojure DSL for creating topologies.
 * Increased the default timeouts for startup of workers and tasks.
 * Added the commands "localconfvalue", "remoteconfvalue", and "repl" to the storm script.
 * Better error message when "storm jar" can't find the nimbus host in the configuration. 

## 0.5.2

 * No longer need any native dependencies to run Storm in local mode. Storm now uses a pure Java messaging system in local mode
 * Fixed logging configurations so that logging is no longer suppressed when including the Storm release jars on the classpath in local mode. 

## 0.5.1

 * Changed ISerialization's "accept" interface to not annotate the Class with the generic type
 * Made Config class implement Map and added helper methods for setting common configs
 
## 0.5.0
 
 * Initial release!


================================================
FILE: KEYS
================================================
pub   2048R/E80B8FFD 2012-03-05 [expires: 2018-03-05]
uid                  P. Taylor Goetz <ptgoetz@gmail.com>
sig 3        E80B8FFD 2012-03-05  P. Taylor Goetz <ptgoetz@gmail.com>

-----BEGIN PGP PUBLIC KEY BLOCK-----
Version: GnuPG/MacGPG2 v2.0.20 (Darwin)
Comment: GPGTools - http://gpgtools.org

mQENBE9VAVUBCADwWjI9USSW4xx45L0KSeHiu+rT1t2eolKx+yxxfMC9QJWb1uGt
WCKG2zb2lk6DBej2/vF6v6EA6d+esOZfmSZazkd61q0INyimuxi0PBHEjipWD/f3
uj87ylGY6WbhQjv60eRlQLMH5Md7zGtzUQGmi7BlogTiwWvcYGvYjmkpk6AyGrE2
9VhJrtRMXpX53V1iL79Z8QR6l688oyuxV3OmPVQMJADtqbXMrDiHk+nSpVuZT5gm
CA3Fl5zfq7RdsPLrJeNDNM+sL0IuKiFX5U2RVuXF3G4BWoBoHtot8ZG01YhKP7gG
/7l2fLd5q/sytCcahT7uLTG/rIC829tFvjMvABEBAAG0I1AuIFRheWxvciBHb2V0
eiA8cHRnb2V0ekBnbWFpbC5jb20+iQE+BBMBAgAoBQJPVQFVAhsvBQkLSIaABgsJ
CAcDAgYVCAIJCgsEFgIDAQIeAQIXgAAKCRCN4Dli6AuP/bqmB/9/U1AzfpMFJ/dY
noqCY2yEYV54Bm6e59qlYUifPEFCMKULB5IzMdyou2DYoUrJquHTYdsHUBTr8cuN
4wVnro8AsryNXjo8oFmE9JwrrO6jE5GLt1OTvri+e0MYgvb08Fk54aZg/zXTcNNS
pIdkbLDBj/RL5jdflKAFuYKSsIEaj0bCvECoR1CRPfTJX2XtPDzRTP28ccRu/pEz
2I588JSZ/RSjqk9DW2Mh75g1CBocRLp90qhW9jUoCkZb0Pis8jnm5gkcHYOz5Hpr
qPzxjZOlMD+cLkP9Geo0+Gs13tt3rwBgIE0l/mPdRltPBbQ9xXORoMlGHtZlXZrn
qSx4e87y
=RfYX
-----END PGP PUBLIC KEY BLOCK-----


================================================
FILE: LICENSE
================================================

                                 Apache License
                           Version 2.0, January 2004
                        http://www.apache.org/licenses/

   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION

   1. Definitions.

      "License" shall mean the terms and conditions for use, reproduction,
      and distribution as defined by Sections 1 through 9 of this document.

      "Licensor" shall mean the copyright owner or entity authorized by
      the copyright owner that is granting the License.

      "Legal Entity" shall mean the union of the acting entity and all
      other entities that control, are controlled by, or are under common
      control with that entity. For the purposes of this definition,
      "control" means (i) the power, direct or indirect, to cause the
      direction or management of such entity, whether by contract or
      otherwise, or (ii) ownership of fifty percent (50%) or more of the
      outstanding shares, or (iii) beneficial ownership of such entity.

      "You" (or "Your") shall mean an individual or Legal Entity
      exercising permissions granted by this License.

      "Source" form shall mean the preferred form for making modifications,
      including but not limited to software source code, documentation
      source, and configuration files.

      "Object" form shall mean any form resulting from mechanical
      transformation or translation of a Source form, including but
      not limited to compiled object code, generated documentation,
      and conversions to other media types.

      "Work" shall mean the work of authorship, whether in Source or
      Object form, made available under the License, as indicated by a
      copyright notice that is included in or attached to the work
      (an example is provided in the Appendix below).

      "Derivative Works" shall mean any work, whether in Source or Object
      form, that is based on (or derived from) the Work and for which the
      editorial revisions, annotations, elaborations, or other modifications
      represent, as a whole, an original work of authorship. For the purposes
      of this License, Derivative Works shall not include works that remain
      separable from, or merely link (or bind by name) to the interfaces of,
      the Work and Derivative Works thereof.

      "Contribution" shall mean any work of authorship, including
      the original version of the Work and any modifications or additions
      to that Work or Derivative Works thereof, that is intentionally
      submitted to Licensor for inclusion in the Work by the copyright owner
      or by an individual or Legal Entity authorized to submit on behalf of
      the copyright owner. For the purposes of this definition, "submitted"
      means any form of electronic, verbal, or written communication sent
      to the Licensor or its representatives, including but not limited to
      communication on electronic mailing lists, source code control systems,
      and issue tracking systems that are managed by, or on behalf of, the
      Licensor for the purpose of discussing and improving the Work, but
      excluding communication that is conspicuously marked or otherwise
      designated in writing by the copyright owner as "Not a Contribution."

      "Contributor" shall mean Licensor and any individual or Legal Entity
      on behalf of whom a Contribution has been received by Licensor and
      subsequently incorporated within the Work.

   2. Grant of Copyright License. Subject to the terms and conditions of
      this License, each Contributor hereby grants to You a perpetual,
      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
      copyright license to reproduce, prepare Derivative Works of,
      publicly display, publicly perform, sublicense, and distribute the
      Work and such Derivative Works in Source or Object form.

   3. Grant of Patent License. Subject to the terms and conditions of
      this License, each Contributor hereby grants to You a perpetual,
      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
      (except as stated in this section) patent license to make, have made,
      use, offer to sell, sell, import, and otherwise transfer the Work,
      where such license applies only to those patent claims licensable
      by such Contributor that are necessarily infringed by their
      Contribution(s) alone or by combination of their Contribution(s)
      with the Work to which such Contribution(s) was submitted. If You
      institute patent litigation against any entity (including a
      cross-claim or counterclaim in a lawsuit) alleging that the Work
      or a Contribution incorporated within the Work constitutes direct
      or contributory patent infringement, then any patent licenses
      granted to You under this License for that Work shall terminate
      as of the date such litigation is filed.

   4. Redistribution. You may reproduce and distribute copies of the
      Work or Derivative Works thereof in any medium, with or without
      modifications, and in Source or Object form, provided that You
      meet the following conditions:

      (a) You must give any other recipients of the Work or
          Derivative Works a copy of this License; and

      (b) You must cause any modified files to carry prominent notices
          stating that You changed the files; and

      (c) You must retain, in the Source form of any Derivative Works
          that You distribute, all copyright, patent, trademark, and
          attribution notices from the Source form of the Work,
          excluding those notices that do not pertain to any part of
          the Derivative Works; and

      (d) If the Work includes a "NOTICE" text file as part of its
          distribution, then any Derivative Works that You distribute must
          include a readable copy of the attribution notices contained
          within such NOTICE file, excluding those notices that do not
          pertain to any part of the Derivative Works, in at least one
          of the following places: within a NOTICE text file distributed
          as part of the Derivative Works; within the Source form or
          documentation, if provided along with the Derivative Works; or,
          within a display generated by the Derivative Works, if and
          wherever such third-party notices normally appear. The contents
          of the NOTICE file are for informational purposes only and
          do not modify the License. You may add Your own attribution
          notices within Derivative Works that You distribute, alongside
          or as an addendum to the NOTICE text from the Work, provided
          that such additional attribution notices cannot be construed
          as modifying the License.

      You may add Your own copyright statement to Your modifications and
      may provide additional or different license terms and conditions
      for use, reproduction, or distribution of Your modifications, or
      for any such Derivative Works as a whole, provided Your use,
      reproduction, and distribution of the Work otherwise complies with
      the conditions stated in this License.

   5. Submission of Contributions. Unless You explicitly state otherwise,
      any Contribution intentionally submitted for inclusion in the Work
      by You to the Licensor shall be under the terms and conditions of
      this License, without any additional terms or conditions.
      Notwithstanding the above, nothing herein shall supersede or modify
      the terms of any separate license agreement you may have executed
      with Licensor regarding such Contributions.

   6. Trademarks. This License does not grant permission to use the trade
      names, trademarks, service marks, or product names of the Licensor,
      except as required for reasonable and customary use in describing the
      origin of the Work and reproducing the content of the NOTICE file.

   7. Disclaimer of Warranty. Unless required by applicable law or
      agreed to in writing, Licensor provides the Work (and each
      Contributor provides its Contributions) on an "AS IS" BASIS,
      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
      implied, including, without limitation, any warranties or conditions
      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
      PARTICULAR PURPOSE. You are solely responsible for determining the
      appropriateness of using or redistributing the Work and assume any
      risks associated with Your exercise of permissions under this License.

   8. Limitation of Liability. In no event and under no legal theory,
      whether in tort (including negligence), contract, or otherwise,
      unless required by applicable law (such as deliberate and grossly
      negligent acts) or agreed to in writing, shall any Contributor be
      liable to You for damages, including any direct, indirect, special,
      incidental, or consequential damages of any character arising as a
      result of this License or out of the use or inability to use the
      Work (including but not limited to damages for loss of goodwill,
      work stoppage, computer failure or malfunction, or any and all
      other commercial damages or losses), even if such Contributor
      has been advised of the possibility of such damages.

   9. Accepting Warranty or Additional Liability. While redistributing
      the Work or Derivative Works thereof, You may choose to offer,
      and charge a fee for, acceptance of support, warranty, indemnity,
      or other liability obligations and/or rights consistent with this
      License. However, in accepting such obligations, You may act only
      on Your own behalf and on Your sole responsibility, not on behalf
      of any other Contributor, and only if You agree to indemnify,
      defend, and hold each Contributor harmless for any liability
      incurred by, or claims asserted against, such Contributor by reason
      of your accepting any such warranty or additional liability.

   END OF TERMS AND CONDITIONS

   APPENDIX: How to apply the Apache License to your work.

      To apply the Apache License to your work, attach the following
      boilerplate notice, with the fields enclosed by brackets "[]"
      replaced with your own identifying information. (Don't include
      the brackets!)  The text should be enclosed in the appropriate
      comment syntax for the file format. We also recommend that a
      file or class name and description of purpose be included on the
      same "printed page" as the copyright notice for easier
      identification within third-party archives.

   Copyright [yyyy] [name of copyright owner]

   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
   You may obtain a copy of the License at

       http://www.apache.org/licenses/LICENSE-2.0

   Unless required by applicable law or agreed to in writing, software
   distributed under the License is distributed on an "AS IS" BASIS,
   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   See the License for the specific language governing permissions and
   limitations under the License.

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


For jQuery 1.6.2 (storm-core/src/ui/public/js/jquery-1.6.2.min.js)

Copyright (c) 2009 John Resig, http://jquery.com/

Permission is hereby granted, free of charge, to any person obtaining
a copy of this software and associated documentation files (the
"Software"), to deal in the Software without restriction, including
without limitation the rights to use, copy, modify, merge, publish,
distribute, sublicense, and/or sell copies of the Software, and to
permit persons to whom the Software is furnished to do so, subject to
the following conditions:

The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.

THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

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

For jQuery Cookies 2.2.0 (storm-core/src/ui/public/js/jquery.cookies.2.2.0.min.js)

Copyright (c) 2005 - 2010, James Auldridge

Permission is hereby granted, free of charge, to any person obtaining
a copy of this software and associated documentation files (the
"Software"), to deal in the Software without restriction, including
without limitation the rights to use, copy, modify, merge, publish,
distribute, sublicense, and/or sell copies of the Software, and to
permit persons to whom the Software is furnished to do so, subject to
the following conditions:

The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.

THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

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

For jQuery TableSorter 2.0.5b (storm-core/src/ui/public/js/jquery.tablesorter.min.js)

Copyright (c) 2007 Christian Bach
Examples and docs at: http://tablesorter.com
Dual licensed under the MIT and GPL licenses:
http://www.opensource.org/licenses/mit-license.php
http://www.gnu.org/licenses/gpl.html

MIT license selected:

Copyright (c) 2007 Christian Bach

Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:

The above copyright notice and this permission notice shall be included in
all copies or substantial portions of the Software.

THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.

================================================
FILE: MODULES
================================================
storm-console-logging
storm-core
storm-netty



================================================
FILE: NOTICE
================================================
Apache Storm
Copyright 2013 The Apache Software Foundation

This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).


This product includes software developed by Nathan Marz
Copyright 2011-2013 Nathan Marz


This product includes software developed by Yahoo! Inc. (www.yahoo.com)
Copyright © 2012-2013 Yahoo! Inc.  All rights reserved.


================================================
FILE: README.markdown
================================================
## IMPORTANT NOTE!!!
Storm has Moved to Apache. The official Storm git repository is now hosted by Apache, and is mirrored on github here:

[https://github.com/apache/incubator-storm](https://github.com/apache/incubator-storm)


### Contributing
Source code contributions can be submitted either by [sumitting a pull request](https://github.com/apache/incubator-storm/pulls) or by creating an issue in [JIRA](https://issues.apache.org/jira/browse/STORM) and attaching patches.

### Migrating Git Repos from nathanmarz/storm to apache/incubator-storm
If you have an existing fork/clone of nathanmarz/storm, you can migrate to apache/incubator-storm by doing the following:

1. Create a new fork of [apache/incubator-storm]()
2. Point your existing clone to the new fork:


		git remote remove origin
		git remote add origin git@github.com:username/incubator-storm.git



### Issue Tracking
The official issue tracker for Storm is Apache JIRA:

[https://issues.apache.org/jira/browse/STORM](https://issues.apache.org/jira/browse/STORM)



### User Mailing List
Storm users should send messages and subscribe to [user@storm.incubator.apache.org](mailto:user@storm.incubator.apache.org).

You can subscribe to this list by sending an email to [user-subscribe@storm.incubator.apache.org](mailto:user-subscribe@storm.incubator.apache.org). Likewise, you can cancel a subscription by sending an email to [user-unsubscribe@storm.incubator.apache.org](mailto:user-unsubscribe@storm.incubator.apache.org).

You can view the archives of the mailing list [here](http://mail-archives.apache.org/mod_mbox/incubator-storm-user/).

### Developer Mailing List
Storm developers should send messages and subscribe to [dev@storm.incubator.apache.org](mailto:dev@storm.incubator.apache.org).

You can subscribe to this list by sending an email to [dev-subscribe@storm.incubator.apache.org](mailto:dev-subscribe@storm.incubator.apache.org). Likewise, you can cancel a subscription by sending an email to [dev-unsubscribe@storm.incubator.apache.org](mailto:dev-unsubscribe@storm.incubator.apache.org).

You can view the archives of the mailing list [here](http://mail-archives.apache.org/mod_mbox/incubator-storm-dev/).

### Which list should I send/subscribe to?
If you are using a pre-built binary distribution of Storm, then chances are you should send questions, comments, storm-related announcements, etc. to [user@storm.apache.incubator.org](user@storm.apache.incubator.org). 

If you are building storm from source, developing new features, or otherwise hacking storm source code, then [dev@storm.incubator.apache.org](dev@storm.incubator.apache.org) is more appropriate. 

### What will happen with storm-user@googlegroups.com?
All existing messages will remain archived there, and can be accessed/searched [here](https://groups.google.com/forum/#!forum/storm-user).

New messages sent to storm-user@googlegroups.com will either be rejected/bounced or replied to with a message to direct the email to the appropriate Apache-hosted group.


================================================
FILE: TODO
================================================
Use cases:

1. number of steps between 2 people in a graph (topology with cycles?)


#################

* Repackage jzmq and zmq as a leiningen "native dep"
       - this might be good, since the native dep can package builds for all different systems/os's?


* Deploy design:

- storm swap {name} {jar} {class}
- it's allowed to use resources equal to current running topology plus number of free resources
- starts in deactivated mode
- add TOPOLOGY_STARTUP_TIME config for the delay until nimbus activates a topology after launching it
- for swap, after the startup time, deactivate the other topology, wait the TOPOLOGY_MESSAGE_TIMEOUT_SECS, and then activate the other topology
- should be able to decrease the message timeout for killing or swapping (add optional thrift parameter) -- or just make it part of the config?
- add killWithOptions, swap, swapWithOptions

* Storm UI, stats, debugging, diagnosis tools
-- need to be able to hide system streams/components from the calculations (another query param and should be default)
-- need to optimize (slowness is probably on nimbus end of querying zk, consider adding heartbeat caching into nimbus)
-- add margins
-- add titles so its easier to distinguish the various pages
-- right align all table columns except for the leftmost
		
* Unit test the core pieces that have stabilized their APIs

- process simulator
- virtual ports
- supervisor
- utils
- test worker/tasks

* implement pseudo-distributed mode -- this is for testing the distributed parts of the code
  - perhaps i can use pallet/vmfest for this

* Need integration tests that run on an actual storm cluster (scp code/process code/zookeeper code not tested in unit tests)

* bolts with none grouping can be pushed into a bolt. e.g. A -> B -> C
     A -> D -> E
     
If A -> B and A -> D are shuffle grouping = none, and B -> C and D -> E are not, then both can be run in A, b's branch goes to C and D's branch goes to E


* Failure design

Add fail method to outputcollector
Fail sends fail message to Acker for those anchors, which sends fail message back to spout.
Whenever spout fails a tuple, it emits it in its failure stream...

Add fail method to drpc... Causes blocked thread to throw exception

* Have worker heartbeat with its task ids, nimbus verifies - if wrong, reassign tasks?
- detect and ignore stray tasks
Each worker can choose a unique id for itself when heart beating
- nimbus deletes those that aren't in topology

* Subscriptions design

-- new kind of spout: "subscription spout"
   --> goal is to sync it's data across the tasks that subscribe to its streams
   --> after doing a grouping, remembers what task it sent the tuple to (regardless of grouping). if a task dies, it knows its subscriptions and asks to be resynced
   --> normal operation is to push to tasks, but pull done when a task starts up (b/c previous task died or something)
   --> need to be able to add tuples to subscription or take tuples away (this is protocol with who you're subscribing to - e.g. rocket)
   --> subscriptions can only happen in a spout because it requires persistent state
   --> when subscription spout task dies, it polls the source (e.g. rocket) for all the subscription info
   --> ideally you'd set things up to have one subscription spout per rocket server
   --> TODO: Need some way to delete subscriptions -> part of tuple or extra metadata on tuple (extra metadata seems cleaner)
        --> add isSubscription() method to Tuple as well as a getSubscriptionType() [which returns ADD or REMOVE]
   --> when a spout starts up, it also needs to push all of its subscription info
   --> acks are irrelevant for subscription tuples -- how should acks be managed as an abstraction?
        -- maybe the synchronized state is done for you -- you just access the state directly and receive a callback whenever it changes?
        -- so don't use tuples...
   --> subscriptions break all the abstractions, perhaps I should generalize spouts and factor acking as a library on top of storm. subscriptions would just be another kind of library? -> no, it seems to break abstractions anyway (like keeping task -> tuples in memory)
   --> maybe call it "syncspout"
   --> if just do syncing (don't expose tuples directly?)
   --> have a "SubscribedState" class that takes care of indexing/etc. --> expose it through topologycontext?
      -- need a way to distinguish between states of different streams
      -- has "add" and "remove" methods
      -- bolt can give a statemanager object that implements add and remove in the prepare method
      -- add(Tuple tuple)
      -- remove(Tuple tuple)
   --> synchronize protocol (when spout or source of data dies):
      --> send how many tuples are going to be sent
      --> send the tuples
      --> OR: pack everything together into a single message (could be hard b/c where tuples are supposed to go is abstracted away)
      --> tie everything together with a unique ID
      --> once task receives everything, has info needed to remove tuples
   --> statespout should do long-polling with timeout
   --> to do subscriptions, the state should contain something like [url, subscriber]. some bolt appends subscriber to tuples, group by subscriber, and send info back
        --> how to to fields grouping with an even distribution?
   -->  ********* tasks need to block on startup until they're synchronized *********
          --> send sync messages in a loop until it's synchronized
          --> add a task.synchronize.poll.freq.secs config (default to 10 seconds)
          --> need to buffer other messages as topology is waiting for synchronization messages (use disk?)
   --> could use acking system to know if a piece of state gets fully synchronized and communicate this with user
      --> perhaps expose this through a special stream? (the state status stream -> similar to failure streams)
   --> should be able to do updates of existing state
      --> use case: have a knob that you can set externally
      --> this isn't really any better than just using zookeeper directly
   
   
_myState = context.setSubscribedState(_myState)

StateSpout {
  //does a timeout long poll and emits new add or remove state tuples (add and remove on the output collector)
  nextTuple(StateSpoutOutputCollector) //collector has add and remove methods add(id, tuple). remove(id)
  //emits all the tuples into the output collector (in the background, will also send ids and counts to tasks so they know how to synchronize)
  //called on startup
  //collector can have a synchronize method in case the source of data (e.g., rocket) craps out
  synchronize(SynchronizationOutputCollector) //collector only has add(id, tuple) method
}

//task startup (in prepare method) [this is automatic]
for(int taskId: statespoutids) {
  emitDirect(SYNC_STREAM, tuple())
}

statespout synchronization():
  id = uuid()
  //getAlLStateTuples calls synchronize on the spout to get the tuples
  for(Tuple t: getAllStateTuplesFromSource()) {
    List tasks = emit(cons(id, t));
    .. keep track of id -> tasks -> count
    for(task: all output tasks) {
      emitDirect(task, id, count)
    } 
  }

for synchronization to work, task needs to keep track of which tasks sent it tuples, and compare against only that set on synchronization

Need a way to propogate information back up the topology - "subscriptions"
e.g. browser -> rocket -> bolt -> bolt -> bolt. 

example: #retweets for a subscribed set of tweet ids

storm topology

 -> tweet spout (A) -> group on original id -> count (B) -> rocket

subscriptions: rocket -> count (B) tweet id (need to group) -> spout (need to go to all)

-- how does it work when stuff dies downstream or upstream? do people ask what the subscriptions are? or do you push your subscriptions up? a combination?

-- maybe subscriptions are a "constant" spout? e..g, continuously emits and refreshes to make sure every task has the tuple. this seem amporphous and hard to implement... nimbus would need to refire all constant spouts whenever there's a reassignment that affects the flow of data. subscriptions seem more natural

-- subscriptions are a special kind of stream that are driven by being asked to send it. e..g, rocket is a spout that emits subscription/unsubscription tuples. they only send it when they get something new, or are asked as to what all the subscriptions are

-- maybe you just need a system stream to know when tasks are created. when you see that a downstream task is created, you know to fire subscriptions to it if its subscribed to your subscriptions stream? - how does this interplay with all the grouping types... you almost want to do a grouping and only send what to tasks that would have received. spouts would need to be able to subscribe to streams as well

(use 'backtype.storm.testing)
;;(start-simulating-time!)
(def cluster (mk-local-storm-cluster))
(use 'backtype.storm.bootstrap) (bootstrap)
(import '[backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
(def spout (feeder-spout ["word"]))
(def topology (thrift/mk-topology
                    {1 (thrift/mk-spout-spec spout :parallelism-hint 3)}
                    {2 (thrift/mk-bolt-spec {1 ["word"]} (TestWordCounter.) :parallelism-hint 4)
                     3 (thrift/mk-bolt-spec {1 :global} (TestGlobalCount.))
                     4 (thrift/mk-bolt-spec {2 :global} (TestAggregatesCounter.))
                     }))
(submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 4 TOPOLOGY-DEBUG true} topology)


* clean up project
  - remove log4j dir and instead generate it in the deploy (it's only used in bin/storm -> create a console one and put into bin/)
  - include system component / stream information in the topologycontext and clean up system specific code all over the place

* Very rare errors

weird nullptr exceptions:
(tasks i) on send-fn
no virtual port socket for outbound task (in worker)



================================================
FILE: VERSION
================================================
0.9.1-incubating-SNAPSHOT


================================================
FILE: bin/build_modules.sh
================================================
#!/bin/bash

# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

LEIN=`which lein2 || which lein` 
export LEIN_ROOT=1


for module in $(cat MODULES)
do
	echo "Building $module"
	cd $module
	if [ $module != "storm-console-logging" ]
		then
			rm ../conf/logback.xml
	fi


	$LEIN with-profile release clean
	$LEIN with-profile release deps
	$LEIN with-profile release jar
	$LEIN with-profile release install
	$LEIN with-profile release pom

	git checkout ../conf/logback.xml
	cd ..
done

================================================
FILE: bin/build_release.sh
================================================
#!/bin/bash

# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

function quit {
    exit 1
}
trap quit 1 2 3 15  #Ctrl+C exits.

RELEASE=`cat VERSION`
LEIN=`which lein2 || which lein` 
export LEIN_ROOT=1

echo Making release $RELEASE

DIR=`pwd`/_release/storm-$RELEASE

rm -rf _release
rm -f *.zip
rm -f *.tar.gz
$LEIN pom || exit 1
mkdir -p $DIR/lib


sh bin/build_modules.sh

for module in $(cat MODULES)
do
	cd $module
	mvn dependency:copy-dependencies || exit 1
	cp -f target/dependency/*.jar $DIR/lib/
	cp -f target/*.jar $DIR/
	cd ..
done

cd _release/storm-$RELEASE
for i in *.jar
do
	rm -f lib/$i
done 
cd ../..

cp CHANGELOG.md $DIR/

echo $RELEASE > $DIR/RELEASE

mkdir -p $DIR/logback
mkdir -p $DIR/logs
cp -R logback/cluster.xml $DIR/logback/cluster.xml

mkdir $DIR/conf
cp conf/storm.yaml.example $DIR/conf/storm.yaml

cp -R storm-core/src/ui/public $DIR/

cp -R bin $DIR/

cp README.markdown $DIR/
cp LICENSE.html $DIR/

cd _release
zip -r storm-$RELEASE.zip *
mv storm-*.zip ../
tar -cvzf ../storm-$RELEASE.tar.gz ./

cd ..

rm -rf _release



================================================
FILE: bin/install_zmq.sh
================================================
#!/bin/bash

# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

export JAVA_HOME=${JAVA_HOME:/usr/libexec/java_home}

if [ ! -d "$JAVA_HOME/include" ]; then
    echo "
Looks like you're missing your 'include' directory. If you're using Mac OS X, You'll need to install the Java dev package.

- Navigate to http://goo.gl/D8lI
- Click the Java tab on the right
- Install the appropriate version and try again.
"
    exit -1;
fi

#install zeromq
wget http://download.zeromq.org/zeromq-2.1.7.tar.gz
tar -xzf zeromq-2.1.7.tar.gz
cd zeromq-2.1.7
./configure
make
sudo make install

cd ../

#install jzmq (both native and into local maven cache)
git clone https://github.com/nathanmarz/jzmq.git
cd jzmq
./autogen.sh
./configure
make
sudo make install


================================================
FILE: bin/javadoc.sh
================================================
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

mkdir -p doc
javadoc -d doc-$1/ `find . -name "*.java" | grep -v generated`


================================================
FILE: bin/storm
================================================
#!/usr/bin/python

# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

import os
import sys
import random
import subprocess as sub
import getopt
import re

def identity(x):
    return x

def cygpath(x):
    command = ["cygpath", "-wp", x]
    p = sub.Popen(command,stdout=sub.PIPE)
    output, errors = p.communicate()
    lines = output.split("\n")
    return lines[0]

if sys.platform == "cygwin":
    normclasspath = cygpath
else:
    normclasspath = identity

STORM_DIR = "/".join(os.path.realpath( __file__ ).split("/")[:-2])
USER_CONF_DIR = os.path.expanduser("~/.storm")
CLUSTER_CONF_DIR = STORM_DIR + "/conf"
if (not os.path.isfile(USER_CONF_DIR + "/storm.yaml")):
    USER_CONF_DIR = CLUSTER_CONF_DIR
CONFIG_OPTS = []
CONFFILE = ""
JAR_JVM_OPTS = os.getenv('STORM_JAR_JVM_OPTS', '')


def get_config_opts():
    global CONFIG_OPTS
    return "-Dstorm.options=" + (','.join(CONFIG_OPTS)).replace(' ', "%%%%")

if not os.path.exists(STORM_DIR + "/RELEASE"):
    print "******************************************"
    print "The storm client can only be run from within a release. You appear to be trying to run the client from a checkout of Storm's source code."
    print "\nYou can download a Storm release at http://storm-project.net/downloads.html"
    print "******************************************"
    sys.exit(1)  

def get_jars_full(adir):
    files = os.listdir(adir)
    ret = []
    for f in files:
        if f.endswith(".jar"):
            ret.append(adir + "/" + f)
    return ret

def get_classpath(extrajars):
    ret = get_jars_full(STORM_DIR)
    ret.extend(get_jars_full(STORM_DIR + "/lib"))
    ret.extend(extrajars)
    return normclasspath(":".join(ret))

def confvalue(name, extrapaths):
    global CONFFILE
    command = [
        "java", "-client", get_config_opts(), "-Dstorm.conf.file=" + CONFFILE, "-cp", get_classpath(extrapaths), "backtype.storm.command.config_value", name
    ]
    p = sub.Popen(command, stdout=sub.PIPE)
    output, errors = p.communicate()
    lines = output.split("\n")
    for line in lines:
        tokens = line.split(" ")
        if tokens[0] == "VALUE:":
            return " ".join(tokens[1:])
    return ""

def print_localconfvalue(name):
    """Syntax: [storm localconfvalue conf-name]

    Prints out the value for conf-name in the local Storm configs. 
    The local Storm configs are the ones in ~/.storm/storm.yaml merged 
    in with the configs in defaults.yaml.
    """
    print name + ": " + confvalue(name, [USER_CONF_DIR])

def print_remoteconfvalue(name):
    """Syntax: [storm remoteconfvalue conf-name]

    Prints out the value for conf-name in the cluster's Storm configs. 
    The cluster's Storm configs are the ones in $STORM-PATH/conf/storm.yaml 
    merged in with the configs in defaults.yaml. 

    This command must be run on a cluster machine.
    """
    print name + ": " + confvalue(name, [CLUSTER_CONF_DIR])

def parse_args(string):
    r"""Takes a string of whitespace-separated tokens and parses it into a list.
    Whitespace inside tokens may be quoted with single quotes, double quotes or
    backslash (similar to command-line arguments in bash).

    >>> parse_args(r'''"a a" 'b b' c\ c "d'd" 'e"e' 'f\'f' "g\"g" "i""i" 'j''j' k" "k l' l' mm n\\n''')
    ['a a', 'b b', 'c c', "d'd", 'e"e', "f'f", 'g"g', 'ii', 'jj', 'k k', 'l l', 'mm', r'n\n']
    """
    re_split = re.compile(r'''((?:
        [^\s"'\\] |
        "(?: [^"\\] | \\.)*" |
        '(?: [^'\\] | \\.)*' |
        \\.
    )+)''', re.VERBOSE)
    args = re_split.split(string)[1::2]
    args = [re.compile(r'"((?:[^"\\]|\\.)*)"').sub('\\1', x) for x in args]
    args = [re.compile(r"'((?:[^'\\]|\\.)*)'").sub('\\1', x) for x in args]
    return [re.compile(r'\\(.)').sub('\\1', x) for x in args]

def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[], fork=False):
    global CONFFILE
    all_args = [
        "java", jvmtype, get_config_opts(),
        "-Dstorm.home=" + STORM_DIR, 
        "-Djava.library.path=" + confvalue("java.library.path", extrajars),
        "-Dstorm.conf.file=" + CONFFILE,
        "-cp", get_classpath(extrajars),
    ] + jvmopts + [klass] + list(args)
    print "Running: " + " ".join(all_args)
    if fork:
        os.spawnvp(os.P_WAIT, "java", all_args)
    else:
        os.execvp("java", all_args) # replaces the current process and never returns

def jar(jarfile, klass, *args):
    """Syntax: [storm jar topology-jar-path class ...]

    Runs the main method of class with the specified arguments. 
    The storm jars and configs in ~/.storm are put on the classpath. 
    The process is configured so that StormSubmitter 
    (http://nathanmarz.github.com/storm/doc/backtype/storm/StormSubmitter.html)
    will upload the jar at topology-jar-path when the topology is submitted.
    """
    exec_storm_class(
        klass,
        jvmtype="-client",
        extrajars=[jarfile, USER_CONF_DIR, STORM_DIR + "/bin"],
        args=args,
        jvmopts=[' '.join(filter(None, [JAR_JVM_OPTS, "-Dstorm.jar=" + jarfile]))])

def kill(*args):
    """Syntax: [storm kill topology-name [-w wait-time-secs]]

    Kills the topology with the name topology-name. Storm will 
    first deactivate the topology's spouts for the duration of 
    the topology's message timeout to allow all messages currently 
    being processed to finish processing. Storm will then shutdown 
    the workers and clean up their state. You can override the length 
    of time Storm waits between deactivation and shutdown with the -w flag.
    """
    exec_storm_class(
        "backtype.storm.command.kill_topology", 
        args=args, 
        jvmtype="-client", 
        extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])

def activate(*args):
    """Syntax: [storm activate topology-name]

    Activates the specified topology's spouts.
    """
    exec_storm_class(
        "backtype.storm.command.activate", 
        args=args, 
        jvmtype="-client", 
        extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])

def listtopos(*args):
    """Syntax: [storm list]

    List the running topologies and their statuses.
    """
    exec_storm_class(
        "backtype.storm.command.list", 
        args=args, 
        jvmtype="-client", 
        extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])

def deactivate(*args):
    """Syntax: [storm deactivate topology-name]

    Deactivates the specified topology's spouts.
    """
    exec_storm_class(
        "backtype.storm.command.deactivate", 
        args=args, 
        jvmtype="-client", 
        extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])

def rebalance(*args):
    """Syntax: [storm rebalance topology-name [-w wait-time-secs] [-n new-num-workers] [-e component=parallelism]*]

    Sometimes you may wish to spread out where the workers for a topology 
    are running. For example, let's say you have a 10 node cluster running 
    4 workers per node, and then let's say you add another 10 nodes to 
    the cluster. You may wish to have Storm spread out the workers for the 
    running topology so that each node runs 2 workers. One way to do this 
    is to kill the topology and resubmit it, but Storm provides a "rebalance" 
    command that provides an easier way to do this.

    Rebalance will first deactivate the topology for the duration of the 
    message timeout (overridable with the -w flag) and then redistribute 
    the workers evenly around the cluster. The topology will then return to 
    its previous state of activation (so a deactivated topology will still 
    be deactivated and an activated topology will go back to being activated).
    
    The rebalance command can also be used to change the parallelism of a running topology.
    Use the -n and -e switches to change the number of workers or number of executors of a component
    respectively.
    """
    exec_storm_class(
        "backtype.storm.command.rebalance", 
        args=args, 
        jvmtype="-client", 
        extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])

def shell(resourcesdir, command, *args):
    tmpjarpath = "stormshell" + str(random.randint(0, 10000000)) + ".jar"
    os.system("jar cf %s %s" % (tmpjarpath, resourcesdir))
    runnerargs = [tmpjarpath, command]
    runnerargs.extend(args)
    exec_storm_class(
        "backtype.storm.command.shell_submission", 
        args=runnerargs, 
        jvmtype="-client", 
        extrajars=[USER_CONF_DIR],
        fork=True)
    os.system("rm " + tmpjarpath)

def repl():
    """Syntax: [storm repl]

    Opens up a Clojure REPL with the storm jars and configuration 
    on the classpath. Useful for debugging.
    """
    cppaths = [CLUSTER_CONF_DIR]
    exec_storm_class("clojure.lang.Repl", jvmtype="-client", extrajars=cppaths)

def nimbus(klass="backtype.storm.daemon.nimbus"):
    """Syntax: [storm nimbus]

    Launches the nimbus daemon. This command should be run under 
    supervision with a tool like daemontools or monit. 

    See Setting up a Storm cluster for more information.
    (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster)
    """
    cppaths = [CLUSTER_CONF_DIR]
    jvmopts = parse_args(confvalue("nimbus.childopts", cppaths)) + [
        "-Dlogfile.name=nimbus.log",
        "-Dlogback.configurationFile=" + STORM_DIR + "/logback/cluster.xml",
    ]
    exec_storm_class(
        klass, 
        jvmtype="-server", 
        extrajars=cppaths, 
        jvmopts=jvmopts)

def supervisor(klass="backtype.storm.daemon.supervisor"):
    """Syntax: [storm supervisor]

    Launches the supervisor daemon. This command should be run 
    under supervision with a tool like daemontools or monit. 

    See Setting up a Storm cluster for more information.
    (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster)
    """
    cppaths = [CLUSTER_CONF_DIR]
    jvmopts = parse_args(confvalue("supervisor.childopts", cppaths)) + [
        "-Dlogfile.name=supervisor.log",
        "-Dlogback.configurationFile=" + STORM_DIR + "/logback/cluster.xml",
    ]
    exec_storm_class(
        klass, 
        jvmtype="-server", 
        extrajars=cppaths, 
        jvmopts=jvmopts)

def ui():
    """Syntax: [storm ui]

    Launches the UI daemon. The UI provides a web interface for a Storm 
    cluster and shows detailed stats about running topologies. This command 
    should be run under supervision with a tool like daemontools or monit. 

    See Setting up a Storm cluster for more information.
    (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster)
    """
    cppaths = [CLUSTER_CONF_DIR]
    jvmopts = parse_args(confvalue("ui.childopts", cppaths)) + [
        "-Dlogfile.name=ui.log",
        "-Dlogback.configurationFile=" + STORM_DIR + "/logback/cluster.xml",
    ]
    exec_storm_class(
        "backtype.storm.ui.core", 
        jvmtype="-server", 
        jvmopts=jvmopts, 
        extrajars=[STORM_DIR, CLUSTER_CONF_DIR])

def logviewer():
    """Syntax: [storm logviewer]

    Launches the log viewer daemon. It provides a web interface for viewing
    storm log files. This command should be run under supervision with a 
    tool like daemontools or monit. 

    See Setting up a Storm cluster for more information.
    (https://github.com/nathanmarz/storm/wiki/Setting-up-a-Storm-cluster)
    """
    cppaths = [CLUSTER_CONF_DIR]
    jvmopts = parse_args(confvalue("logviewer.childopts", cppaths)) + [
        "-Dlogfile.name=logviewer.log",
        "-Dlogback.configurationFile=" + STORM_DIR + "/logback/cluster.xml",
    ]
    exec_storm_class(
        "backtype.storm.daemon.logviewer", 
        jvmtype="-server", 
        jvmopts=jvmopts, 
        extrajars=[STORM_DIR, CLUSTER_CONF_DIR])

def drpc():
    """Syntax: [storm drpc]

    Launches a DRPC daemon. This command should be run under supervision 
    with a tool like daemontools or monit. 

    See Distributed RPC for more information.
    (https://github.com/nathanmarz/storm/wiki/Distributed-RPC)
    """
    cppaths = [CLUSTER_CONF_DIR]
    jvmopts = parse_args(confvalue("drpc.childopts", cppaths)) + [
        "-Dlogfile.name=drpc.log",
        "-Dlogback.configurationFile=" + STORM_DIR + "/logback/cluster.xml"
    ]
    exec_storm_class(
        "backtype.storm.daemon.drpc", 
        jvmtype="-server", 
        jvmopts=jvmopts, 
        extrajars=[CLUSTER_CONF_DIR])

def dev_zookeeper():
    """Syntax: [storm dev-zookeeper]
  
    Launches a fresh Zookeeper server using "dev.zookeeper.path" as its local dir and
    "storm.zookeeper.port" as its port. This is only intended for development/testing, the
    Zookeeper instance launched is not configured to be used in production.
    """
    cppaths = [CLUSTER_CONF_DIR]
    exec_storm_class(
        "backtype.storm.command.dev_zookeeper", 
        jvmtype="-server", 
        extrajars=[CLUSTER_CONF_DIR])

def version():
  """Syntax: [storm version]
  
  Prints the version number of this Storm release.  
  """
  releasefile = STORM_DIR + "/RELEASE"
  if os.path.exists(releasefile):
    print open(releasefile).readline().strip()
  else:
    print "Unknown"

def print_classpath():
    """Syntax: [storm classpath]

    Prints the classpath used by the storm client when running commands.
    """
    print get_classpath([])

def print_commands():
    """Print all client commands and link to documentation"""
    print "Commands:\n\t",  "\n\t".join(sorted(COMMANDS.keys()))
    print "\nHelp:", "\n\thelp", "\n\thelp <command>"
    print "\nDocumentation for the storm client can be found at https://github.com/nathanmarz/storm/wiki/Command-line-client\n"
    print "Configs can be overridden using one or more -c flags, e.g. \"storm list -c nimbus.host=nimbus.mycompany.com\"\n"

def print_usage(command=None):
    """Print one help message or list of available commands"""
    if command != None:
        if COMMANDS.has_key(command):
            print (COMMANDS[command].__doc__ or 
                  "No documentation provided for <%s>" % command)
        else:
           print "<%s> is not a valid command" % command
    else:
        print_commands()

def unknown_command(*args):
    print "Unknown command: [storm %s]" % ' '.join(sys.argv[1:])
    print_usage()

COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui, "logviewer": logviewer,
            "drpc": drpc, "supervisor": supervisor, "localconfvalue": print_localconfvalue,
            "remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath,
            "activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage,
            "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version}

def parse_config(config_list):
    global CONFIG_OPTS
    if len(config_list) > 0:
        for config in config_list:
            CONFIG_OPTS.append(config)

def parse_config_opts(args):
  curr = args[:]
  curr.reverse()
  config_list = []
  args_list = []
  
  while len(curr) > 0:
    token = curr.pop()
    if token == "-c":
      config_list.append(curr.pop())
    elif token == "--config":
      global CONFFILE
      CONFFILE = curr.pop()
    else:
      args_list.append(token)
  
  return config_list, args_list
    
def main():
    if len(sys.argv) <= 1:
        print_usage()
        sys.exit(-1)
    global CONFIG_OPTS
    config_list, args = parse_config_opts(sys.argv[1:])
    parse_config(config_list)
    COMMAND = args[0]
    ARGS = args[1:]
    (COMMANDS.get(COMMAND, unknown_command))(*ARGS)
    
if __name__ == "__main__":
    main()


================================================
FILE: bin/to_maven.sh
================================================
#!/bin/bash 

# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

function quit {
    exit 1
}
trap quit 1 2 3 15  #Ctrl+C exits.

RELEASE=`cat VERSION`
LEIN=`which lein2 || which lein` 
export LEIN_ROOT=1


sh bin/build_modules.sh

echo ==== Module jars ====
for module in $(cat MODULES)
do
	cd $module
	scp target/*jar pom.xml clojars@clojars.org:
	cd ..
done


#L

echo ==== Storm jar ====
$LEIN clean
$LEIN pom
$LEIN jar
scp pom.xml target/*jar clojars@clojars.org:

echo ==== Storm-lib jar ====
cd storm-lib
$LEIN clean
$LEIN pom
$LEIN jar
scp pom.xml target/*jar clojars@clojars.org:
cd ..


================================================
FILE: conf/defaults.yaml
================================================
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.


########### These all have default values as shown
########### Additional configuration goes into storm.yaml

java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib"

### storm.* configs are general configurations
# the local dir is where jars are kept
storm.local.dir: "storm-local"
storm.zookeeper.servers:
    - "localhost"
storm.zookeeper.port: 2181
storm.zookeeper.root: "/storm"
storm.zookeeper.session.timeout: 20000
storm.zookeeper.connection.timeout: 15000
storm.zookeeper.retry.times: 5
storm.zookeeper.retry.interval: 1000
storm.zookeeper.retry.intervalceiling.millis: 30000
storm.cluster.mode: "distributed" # can be distributed or local
storm.local.mode.zmq: false
storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
storm.messaging.transport: "backtype.storm.messaging.zmq"

### nimbus.* configs are for the master
nimbus.host: "localhost"
nimbus.thrift.port: 6627
nimbus.childopts: "-Xmx1024m"
nimbus.task.timeout.secs: 30
nimbus.supervisor.timeout.secs: 60
nimbus.monitor.freq.secs: 10
nimbus.cleanup.inbox.freq.secs: 600
nimbus.inbox.jar.expiration.secs: 3600
nimbus.task.launch.secs: 120
nimbus.reassign: true
nimbus.file.copy.expiration.secs: 600
nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator"

### ui.* configs are for the master
ui.port: 8080
ui.childopts: "-Xmx768m"

logviewer.port: 8000
logviewer.childopts: "-Xmx128m"
logviewer.appender.name: "A1"


drpc.port: 3772
drpc.worker.threads: 64
drpc.queue.size: 128
drpc.invocations.port: 3773
drpc.request.timeout.secs: 600
drpc.childopts: "-Xmx768m"

transactional.zookeeper.root: "/transactional"
transactional.zookeeper.servers: null
transactional.zookeeper.port: null

### supervisor.* configs are for node supervisors
# Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication
supervisor.slots.ports:
    - 6700
    - 6701
    - 6702
    - 6703
supervisor.childopts: "-Xmx256m"
#how long supervisor will wait to ensure that a worker process is started
supervisor.worker.start.timeout.secs: 120
#how long between heartbeats until supervisor considers that worker dead and tries to restart it
supervisor.worker.timeout.secs: 30
#how frequently the supervisor checks on the status of the processes it's monitoring and restarts if necessary
supervisor.monitor.frequency.secs: 3
#how frequently the supervisor heartbeats to the cluster state (for nimbus)
supervisor.heartbeat.frequency.secs: 5
supervisor.enable: true

### worker.* configs are for task workers
worker.childopts: "-Xmx768m"
worker.heartbeat.frequency.secs: 1

task.heartbeat.frequency.secs: 3
task.refresh.poll.secs: 10

zmq.threads: 1
zmq.linger.millis: 5000
zmq.hwm: 0


storm.messaging.netty.server_worker_threads: 1
storm.messaging.netty.client_worker_threads: 1
storm.messaging.netty.buffer_size: 5242880 #5MB buffer
storm.messaging.netty.max_retries: 30
storm.messaging.netty.max_wait_ms: 1000
storm.messaging.netty.min_wait_ms: 100

### topology.* configs are for specific executing storms
topology.enable.message.timeouts: true
topology.debug: false
topology.optimize: true
topology.workers: 1
topology.acker.executors: null
topology.tasks: null
# maximum amount of time a message has to complete before it's considered failed
topology.message.timeout.secs: 30
topology.skip.missing.kryo.registrations: false
topology.max.task.parallelism: null
topology.max.spout.pending: null
topology.state.synchronization.timeout.secs: 60
topology.stats.sample.rate: 0.05
topology.builtin.metrics.bucket.size.secs: 60
topology.fall.back.on.java.serialization: true
topology.worker.childopts: null
topology.executor.receive.buffer.size: 1024 #batched
topology.executor.send.buffer.size: 1024 #individual messages
topology.receiver.buffer.size: 8 # setting it too high causes a lot of problems (heartbeat thread gets starved, throughput plummets)
topology.transfer.buffer.size: 1024 # batched
topology.tick.tuple.freq.secs: null
topology.worker.shared.thread.pool.size: 4
topology.disruptor.wait.strategy: "com.lmax.disruptor.BlockingWaitStrategy"
topology.spout.wait.strategy: "backtype.storm.spout.SleepSpoutWaitStrategy"
topology.sleep.spout.wait.strategy.time.ms: 1
topology.error.throttle.interval.secs: 10
topology.max.error.report.per.interval: 5
topology.kryo.factory: "backtype.storm.serialization.DefaultKryoFactory"
topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSerializer"
topology.trident.batch.emit.interval.millis: 500

dev.zookeeper.path: "/tmp/dev-storm-zookeeper"


================================================
FILE: conf/jaas_digest.conf
================================================
/*
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you under the Apache License, Version 2.0 (the
 * "License"); you may not use this file except in compliance
 * with the License.  You may obtain a copy of the License at
 *
 * http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

/* This is example of JAAS Login configuration for digest authentication
*/

/* 
StormServer section should contain a list of authorized users and their passwords. 
*/
StormServer {
       org.apache.zookeeper.server.auth.DigestLoginModule required
       user_super="adminsecret"
       user_bob="bobsecret";
       user_john="johnsecret";
};

/*
StormClient section contains one user name and his/her password.
*/
StormClient {
       org.apache.zookeeper.server.auth.DigestLoginModule required
       username="bob"
       password="bobsecret";
};

================================================
FILE: conf/logback.xml
================================================
<?xml version="1.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.
-->
<configuration scan="true" scanPeriod="30 seconds">
  <appender name="A1" class="ch.qos.logback.core.ConsoleAppender">
    <encoder>
      <pattern>%-4r [%t] %-5p %c - %m%n</pattern>
    </encoder>
  </appender>
  <logger name="org.apache.zookeeper" level="WARN"/>
  <root level="INFO">
    <appender-ref ref="A1"/>
  </root>
</configuration>


================================================
FILE: conf/storm.yaml.example
================================================
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
# http:# www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

########### These MUST be filled in for a storm configuration
# storm.zookeeper.servers:
#     - "server1"
#     - "server2"
# 
# nimbus.host: "nimbus"
# 
# 
# ##### These may optionally be filled in:
#    
## List of custom serializations
# topology.kryo.register:
#     - org.mycompany.MyType
#     - org.mycompany.MyType2: org.mycompany.MyType2Serializer
#
## List of custom kryo decorators
# topology.kryo.decorators:
#     - org.mycompany.MyDecorator
#
## Locations of the drpc servers
# drpc.servers:
#     - "server1"
#     - "server2"

## Metrics Consumers
# topology.metrics.consumer.register:
#   - class: "backtype.storm.metrics.LoggingMetricsConsumer"
#     parallelism.hint: 1
#   - class: "org.mycompany.MyMetricsConsumer"
#     parallelism.hint: 1
#     argument:
#       - endpoint: "metrics-collector.mycompany.org"


================================================
FILE: logback/cluster.xml
================================================
<?xml version="1.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.
-->

<configuration scan="true" scanPeriod="60 seconds">
 <appender name="A1" class="ch.qos.logback.core.rolling.RollingFileAppender">
    <file>${storm.home}/logs/${logfile.name}</file>
    <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
      <fileNamePattern>${storm.home}/logs/${logfile.name}.%i</fileNamePattern>
      <minIndex>1</minIndex>
      <maxIndex>9</maxIndex>
    </rollingPolicy>

    <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
      <maxFileSize>100MB</maxFileSize>
    </triggeringPolicy>

    <encoder>
      <pattern>%d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n</pattern>
    </encoder>
 </appender>

 <appender name="ACCESS" class="ch.qos.logback.core.rolling.RollingFileAppender">
    <file>${storm.home}/logs/access.log</file>
    <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
      <fileNamePattern>${storm.home}/logs/access.log.%i</fileNamePattern>
      <minIndex>1</minIndex>
      <maxIndex>9</maxIndex>
    </rollingPolicy>

    <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
      <maxFileSize>100MB</maxFileSize>
    </triggeringPolicy>

    <encoder>
      <pattern>%d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n</pattern>
    </encoder>
  </appender>

  <appender name="METRICS" class="ch.qos.logback.core.rolling.RollingFileAppender">
    <file>${storm.home}/logs/metrics.log</file>
    <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
      <fileNamePattern>metrics.log.%i</fileNamePattern>
      <minIndex>1</minIndex>
      <maxIndex>9</maxIndex>
    </rollingPolicy>

    <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
      <maxFileSize>2MB</maxFileSize>
    </triggeringPolicy>

    <encoder>
      <pattern>%d %-8r %m%n</pattern>
    </encoder>
  </appender>

  <root level="INFO">
    <appender-ref ref="A1"/>
  </root>

  <logger name="backtype.storm.security.auth.authorizer" additivity="false">
    <level value="INFO" />
    <appender-ref ref="ACCESS" />
  </logger>

  <logger name="backtype.storm.metric.LoggingMetricsConsumer" additivity="false" >
    <level value="INFO"/>
    <appender-ref ref="METRICS"/>
  </logger>

</configuration>


================================================
FILE: project.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(def VERSION (.trim (slurp "VERSION")))
(def MODULES (-> "MODULES" slurp (.split "\n")))
(def DEPENDENCIES (for [m MODULES] [(symbol (str "storm/" m)) VERSION]))

(eval `(defproject storm/storm ~VERSION
  :url "http://storm-project.net"
  :description "Distributed and fault-tolerant realtime computation"
  :license {:name "Eclipse Public License - Version 1.0" :url "https://github.com/nathanmarz/storm/blob/master/LICENSE.html"}
  :mailing-list {:name "Storm user mailing list"
                 :archive "https://groups.google.com/group/storm-user"
                 :post "storm-user@googlegroups.com"}
  :dependencies [~@DEPENDENCIES]
  :plugins [[~'lein-sub "0.2.1"]]  
  :min-lein-version "2.0.0"
  :sub [~@MODULES]
  ))


================================================
FILE: storm-console-logging/project.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(def ROOT-DIR (subs *file* 0 (- (count *file*) (count "project.clj"))))
(def VERSION (-> ROOT-DIR (str "/../VERSION") slurp (.trim)))

(defproject storm/storm-console-logging VERSION
  :resource-paths ["logback"]
  :target-path "target"
  :javac-options ["-target" "1.6" "-source" "1.6"]
  :profiles {:release {}
             }

  :aot :all)


================================================
FILE: storm-core/project.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(def ROOT-DIR (subs *file* 0 (- (count *file*) (count "project.clj"))))
(def VERSION (-> ROOT-DIR (str "/../VERSION") slurp (.trim)))

(defproject storm/storm-core VERSION
  :dependencies [[org.clojure/clojure "1.4.0"]
                 [commons-io "1.4"]
                 [org.apache.commons/commons-exec "1.1"]
                 [storm/libthrift7 "0.7.0-2"
                  :exclusions [org.slf4j/slf4j-api]]
                 [clj-time "0.4.1"]
                 [com.netflix.curator/curator-framework "1.0.1"
                  :exclusions [log4j/log4j]]
                 [backtype/jzmq "2.1.0"]
                 [com.googlecode.json-simple/json-simple "1.1"]
                 [compojure "1.1.3"]
                 [hiccup "0.3.6"]
                 [ring/ring-devel "0.3.11"]
                 [ring/ring-jetty-adapter "0.3.11"]
                 [org.clojure/tools.logging "0.2.3"]
                 [org.clojure/math.numeric-tower "0.0.1"]
                 [com.twitter/carbonite "1.3.2"]
                 [org.yaml/snakeyaml "1.11"]
                 [org.apache.httpcomponents/httpclient "4.1.1"]
                 [org.clojure/tools.cli "0.2.2"]
                 [com.googlecode.disruptor/disruptor "2.10.1"]
                 [org.jgrapht/jgrapht-core "0.9.0"]
                 [com.google.guava/guava "13.0"]
                 [ch.qos.logback/logback-classic "1.0.6"]
                 [org.slf4j/log4j-over-slf4j "1.6.6"]
                 ]

  :source-paths ["src/clj"]
  :java-source-paths ["src/jvm"]
  :test-paths ["test/clj"]
  :resource-paths ["../conf"]
  :target-path "target"
  :javac-options ["-target" "1.6" "-source" "1.6"]
  :profiles {:dev {:resource-paths ["src/dev"]
                   :dependencies [[org.mockito/mockito-all "1.9.5"]]}
             :release {}
             :lib {}
             }

  :plugins [[lein-swank "1.4.4"]]

  :repositories {"sonatype"
                 "http://oss.sonatype.org/content/groups/public/"}

  :javac-options ["-g"]
  :jvm-opts ["-Djava.library.path=/usr/local/lib:/opt/local/lib:/usr/lib"]

  :aot :all)


================================================
FILE: storm-core/src/clj/backtype/storm/LocalCluster.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.LocalCluster
  (:use [backtype.storm testing config])
  (:import [java.util Map])
  (:gen-class
   :init init
   :implements [backtype.storm.ILocalCluster]
   :constructors {[] [] [java.util.Map] []}
   :state state ))

(defn -init
  ([]
     (let [ret (mk-local-storm-cluster :daemon-conf {TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true})]
       [[] ret]
       ))
  ([^Map stateMap]
     [[] stateMap]))

(defn -submitTopology [this name conf topology]
  (submit-local-topology (:nimbus (. this state))
                      name
                      conf
                      topology))

(defn -submitTopologyWithOpts [this name conf topology submit-opts]
  (submit-local-topology-with-opts (:nimbus (. this state))
                      name
                      conf
                      topology
                      submit-opts))

(defn -shutdown [this]
  (kill-local-storm-cluster (. this state)))

(defn -killTopology [this name]
  (.killTopology (:nimbus (. this state)) name))

(defn -getTopologyConf [this id]
  (.getTopologyConf (:nimbus (. this state)) id))

(defn -getTopology [this id]
  (.getTopology (:nimbus (. this state)) id))

(defn -getClusterInfo [this]
  (.getClusterInfo (:nimbus (. this state))))

(defn -getTopologyInfo [this id]
  (.getTopologyInfo (:nimbus (. this state)) id))

(defn -killTopologyWithOpts [this name opts]
  (.killTopologyWithOpts (:nimbus (. this state)) name opts))

(defn -activate [this name]
  (.activate (:nimbus (. this state)) name))

(defn -deactivate [this name]
  (.deactivate (:nimbus (. this state)) name))

(defn -rebalance [this name opts]
  (.rebalance (:nimbus (. this state)) name opts))

(defn -getState [this]
  (.state this))



================================================
FILE: storm-core/src/clj/backtype/storm/LocalDRPC.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.LocalDRPC
  (:require [backtype.storm.daemon [drpc :as drpc]])
  (:use [backtype.storm util])
  (:import [backtype.storm.utils InprocMessaging ServiceRegistry])
  (:gen-class
   :init init
   :implements [backtype.storm.ILocalDRPC]
   :constructors {[] []}
   :state state ))

(defn -init []
  (let [handler (drpc/service-handler)
        id (ServiceRegistry/registerService handler)
        ]
    [[] {:service-id id :handler handler}]
    ))

(defn -execute [this func funcArgs]
  (.execute (:handler (. this state)) func funcArgs)
  )

(defn -result [this id result]
  (.result (:handler (. this state)) id result)
  )

(defn -fetchRequest [this func]
  (.fetchRequest (:handler (. this state)) func)
  )

(defn -failRequest [this id]
  (.failRequest (:handler (. this state)) id)
  )
  
(defn -getServiceId [this]
  (:service-id (. this state)))  

(defn -shutdown [this]
  (ServiceRegistry/unregisterService (:service-id (. this state)))
  (.shutdown (:handler (. this state)))
  )


================================================
FILE: storm-core/src/clj/backtype/storm/bootstrap.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.bootstrap)

(defmacro bootstrap []
  '(do
     (import (quote [backtype.storm Constants]))
     (import (quote [backtype.storm.testing FeederSpout TestPlannerBolt TestPlannerSpout
                     AckFailDelegate AckTracker]))
     (import (quote [backtype.storm.utils Utils LocalState Time TimeCacheMap
                     TimeCacheMap$ExpiredCallback
                     RotatingMap RotatingMap$ExpiredCallback
                     BufferFileInputStream
                     RegisteredGlobalState ThriftTopologyUtils DisruptorQueue
                     MutableObject MutableLong]))
     (import (quote [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer]))
     (import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector ShellSpout]))
     (import (quote [backtype.storm.tuple Tuple TupleImpl Fields MessageId]))
     (import (quote [backtype.storm.task IBolt IOutputCollector
                     OutputCollector TopologyContext ShellBolt
                     GeneralTopologyContext WorkerTopologyContext]))
     (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs 
                     IBatchBolt BatchBoltExecutor]))
     (import (quote [backtype.storm.drpc KeyedFairBolt]))
     (import (quote [backtype.storm.daemon Shutdownable]))
     (require (quote [backtype.storm.messaging.loader :as msg-loader]))
     (use (quote [backtype.storm config util log clojure timer]))
     (require (quote [backtype.storm [thrift :as thrift] [cluster :as cluster]
                      [event :as event] [process-simulator :as psim]]))
     (require (quote [clojure.set :as set]))
     (require (quote [backtype.storm [stats :as stats] [disruptor :as disruptor]]))
     (import (quote [org.slf4j Logger]))

     (import (quote [com.lmax.disruptor InsufficientCapacityException]))
     (import (quote [backtype.storm.generated Nimbus Nimbus$Processor
                     Nimbus$Iface StormTopology ShellComponent
                     NotAliveException AlreadyAliveException GlobalStreamId
                     InvalidTopologyException ClusterSummary TopologyInfo
                     TopologySummary ExecutorSummary ExecutorStats ExecutorSpecificStats
                     SpoutStats BoltStats ErrorInfo SupervisorSummary ExecutorInfo
                     KillOptions SubmitOptions RebalanceOptions JavaObject JavaObjectArg
                     TopologyInitialStatus]))
     (import (quote [backtype.storm.daemon.common StormBase Assignment
                     SupervisorInfo WorkerHeartbeat]))
     (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]))
     (import (quote [org.apache.commons.io FileUtils]))
     ))


================================================
FILE: storm-core/src/clj/backtype/storm/clojure.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.clojure
  (:use [backtype.storm bootstrap util])
  (:import [backtype.storm StormSubmitter])
  (:import [backtype.storm.generated StreamInfo])
  (:import [backtype.storm.tuple Tuple])
  (:import [backtype.storm.task OutputCollector IBolt TopologyContext])
  (:import [backtype.storm.spout SpoutOutputCollector ISpout])
  (:import [backtype.storm.utils Utils])
  (:import [backtype.storm.clojure ClojureBolt ClojureSpout])
  (:import [java.util List])
  (:require [backtype.storm [thrift :as thrift]]))


(defn direct-stream [fields]
  (StreamInfo. fields true))

(defn to-spec [avar]
  (let [m (meta avar)]
    [(str (:ns m)) (str (:name m))]))

(defn clojure-bolt* [output-spec fn-var conf-fn-var args]
  (ClojureBolt. (to-spec fn-var) (to-spec conf-fn-var) args (thrift/mk-output-spec output-spec)))

(defmacro clojure-bolt [output-spec fn-sym conf-fn-sym args]
  `(clojure-bolt* ~output-spec (var ~fn-sym) (var ~conf-fn-sym) ~args))

(defn clojure-spout* [output-spec fn-var conf-var args]
  (let [m (meta fn-var)]
    (ClojureSpout. (to-spec fn-var) (to-spec conf-var) args (thrift/mk-output-spec output-spec))
    ))

(defmacro clojure-spout [output-spec fn-sym conf-sym args]
  `(clojure-spout* ~output-spec (var ~fn-sym) (var ~conf-sym) ~args))

(defn normalize-fns [body]
  (for [[name args & impl] body
        :let [args (-> "this"
                       gensym
                       (cons args)
                       vec)]]
    (concat [name args] impl)
    ))

(defmacro bolt [& body]
  (let [[bolt-fns other-fns] (split-with #(not (symbol? %)) body)
        fns (normalize-fns bolt-fns)]
    `(reify IBolt
       ~@fns
       ~@other-fns)))

(defmacro bolt-execute [& body]
  `(bolt
     (~'execute ~@body)))

(defmacro spout [& body]
  (let [[spout-fns other-fns] (split-with #(not (symbol? %)) body)
        fns (normalize-fns spout-fns)]
    `(reify ISpout
       ~@fns
       ~@other-fns)))

(defmacro defbolt [name output-spec & [opts & impl :as all]]
  (if-not (map? opts)
    `(defbolt ~name ~output-spec {} ~@all)
    (let [worker-name (symbol (str name "__"))
          conf-fn-name (symbol (str name "__conf__"))
          params (:params opts)
          conf-code (:conf opts)
          fn-body (if (:prepare opts)
                    (cons 'fn impl)
                    (let [[args & impl-body] impl
                          coll-sym (nth args 1)
                          args (vec (take 1 args))
                          prepargs [(gensym "conf") (gensym "context") coll-sym]]
                      `(fn ~prepargs (bolt (~'execute ~args ~@impl-body)))))
          definer (if params
                    `(defn ~name [& args#]
                       (clojure-bolt ~output-spec ~worker-name ~conf-fn-name args#))
                    `(def ~name
                       (clojure-bolt ~output-spec ~worker-name ~conf-fn-name []))
                    )
          ]
      `(do
         (defn ~conf-fn-name ~(if params params [])
           ~conf-code
           )
         (defn ~worker-name ~(if params params [])
           ~fn-body
           )
         ~definer
         ))))

(defmacro defspout [name output-spec & [opts & impl :as all]]
  (if-not (map? opts)
    `(defspout ~name ~output-spec {} ~@all)
    (let [worker-name (symbol (str name "__"))
          conf-fn-name (symbol (str name "__conf__"))
          params (:params opts)
          conf-code (:conf opts)
          prepare? (:prepare opts)
          prepare? (if (nil? prepare?) true prepare?)
          fn-body (if prepare?
                    (cons 'fn impl)
                    (let [[args & impl-body] impl
                          coll-sym (first args)
                          prepargs [(gensym "conf") (gensym "context") coll-sym]]
                      `(fn ~prepargs (spout (~'nextTuple [] ~@impl-body)))))
          definer (if params
                    `(defn ~name [& args#]
                       (clojure-spout ~output-spec ~worker-name ~conf-fn-name args#))
                    `(def ~name
                       (clojure-spout ~output-spec ~worker-name ~conf-fn-name []))
                    )
          ]
      `(do
         (defn ~conf-fn-name ~(if params params [])
           ~conf-code
           )
         (defn ~worker-name ~(if params params [])
           ~fn-body
           )
         ~definer
         ))))

(defprotocol TupleValues
  (tuple-values [values collector stream]))

(extend-protocol TupleValues
  java.util.Map
  (tuple-values [this collector ^String stream]
    (let [^TopologyContext context (:context collector)
          fields (..  context (getThisOutputFields stream) toList) ]
      (vec (map (into 
                  (empty this) (for [[k v] this] 
                                   [(if (keyword? k) (name k) k) v])) 
                fields))))
  java.util.List
  (tuple-values [this collector stream]
    this))

(defnk emit-bolt! [collector values
                   :stream Utils/DEFAULT_STREAM_ID :anchor []]
  (let [^List anchor (collectify anchor)
        values (tuple-values values collector stream) ]
    (.emit ^OutputCollector (:output-collector collector) stream anchor values)
    ))

(defnk emit-direct-bolt! [collector task values
                          :stream Utils/DEFAULT_STREAM_ID :anchor []]
  (let [^List anchor (collectify anchor)
        values (tuple-values values collector stream) ]
    (.emitDirect ^OutputCollector (:output-collector collector) task stream anchor values)
    ))

(defn ack! [collector ^Tuple tuple]
  (.ack ^OutputCollector (:output-collector collector) tuple))

(defn fail! [collector ^Tuple tuple]
  (.fail ^OutputCollector (:output-collector collector) tuple))

(defn report-error! [collector ^Tuple tuple]
  (.reportError ^OutputCollector (:output-collector collector) tuple))

(defnk emit-spout! [collector values
                    :stream Utils/DEFAULT_STREAM_ID :id nil]
  (let [values (tuple-values values collector stream)]
    (.emit ^SpoutOutputCollector (:output-collector collector) stream values id)))

(defnk emit-direct-spout! [collector task values
                           :stream Utils/DEFAULT_STREAM_ID :id nil]
  (let [values (tuple-values values collector stream)]
    (.emitDirect ^SpoutOutputCollector (:output-collector collector) task stream values id)))

(defalias topology thrift/mk-topology)
(defalias bolt-spec thrift/mk-bolt-spec)
(defalias spout-spec thrift/mk-spout-spec)
(defalias shell-bolt-spec thrift/mk-shell-bolt-spec)
(defalias shell-spout-spec thrift/mk-shell-spout-spec)

(defn submit-remote-topology [name conf topology]
  (StormSubmitter/submitTopology name conf topology))

(defn local-cluster []  
  ;; do this to avoid a cyclic dependency of
  ;; LocalCluster -> testing -> nimbus -> bootstrap -> clojure -> LocalCluster
  (eval '(new backtype.storm.LocalCluster)))


================================================
FILE: storm-core/src/clj/backtype/storm/cluster.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.cluster
  (:import [org.apache.zookeeper.data Stat])
  (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException])
  (:import [backtype.storm.utils Utils])
  (:use [backtype.storm util log config])
  (:require [backtype.storm [zookeeper :as zk]])
  (:require [backtype.storm.daemon [common :as common]])
  
  )

(defprotocol ClusterState
  (set-ephemeral-node [this path data])
  (delete-node [this path])
  (create-sequential [this path data])
  (set-data [this path data])  ;; if node does not exist, create persistent with this data 
  (get-data [this path watch?])
  (get-children [this path watch?])
  (mkdirs [this path])
  (close [this])
  (register [this callback])
  (unregister [this id])
  )

(defn mk-distributed-cluster-state [conf]
  (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf conf)]
    (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT))
    (.close zk))
  (let [callbacks (atom {})
        active (atom true)
        zk (zk/mk-client conf
                         (conf STORM-ZOOKEEPER-SERVERS)
                         (conf STORM-ZOOKEEPER-PORT)
                         :auth-conf conf
                         :root (conf STORM-ZOOKEEPER-ROOT)
                         :watcher (fn [state type path]
                                     (when @active
                                       (when-not (= :connected state)
                                         (log-warn "Received event " state ":" type ":" path " with disconnected Zookeeper."))
                                       (when-not (= :none type)
                                         (doseq [callback (vals @callbacks)]
                                           (callback type path))))
                                       ))]
    (reify
     ClusterState
     (register [this callback]
               (let [id (uuid)]
                 (swap! callbacks assoc id callback)
                 id
                 ))
     (unregister [this id]
                 (swap! callbacks dissoc id))

     (set-ephemeral-node [this path data]
                         (zk/mkdirs zk (parent-path path))
                         (if (zk/exists zk path false)
                           (try-cause
                             (zk/set-data zk path data) ; should verify that it's ephemeral
                             (catch KeeperException$NoNodeException e
                               (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
                               (zk/create-node zk path data :ephemeral)
                               ))
                           (zk/create-node zk path data :ephemeral)
                           ))
     
     (create-sequential [this path data]
       (zk/create-node zk path data :sequential))
     
     (set-data [this path data]
               ;; note: this does not turn off any existing watches
               (if (zk/exists zk path false)
                 (zk/set-data zk path data)
                 (do
                   (zk/mkdirs zk (parent-path path))
                   (zk/create-node zk path data :persistent)
                   )))
     
     (delete-node [this path]
                  (zk/delete-recursive zk path)
                  )
     
     (get-data [this path watch?]
               (zk/get-data zk path watch?)
               )
     
     (get-children [this path watch?]
                   (zk/get-children zk path watch?))
     
     (mkdirs [this path]
             (zk/mkdirs zk path))
     
     (close [this]
            (reset! active false)
            (.close zk))
     )))

(defprotocol StormClusterState
  (assignments [this callback])
  (assignment-info [this storm-id callback])
  (active-storms [this])
  (storm-base [this storm-id callback])

  (get-worker-heartbeat [this storm-id node port])
  (executor-beats [this storm-id executor->node+port])
  (supervisors [this callback])
  (supervisor-info [this supervisor-id])  ;; returns nil if doesn't exist

  (setup-heartbeats! [this storm-id])
  (teardown-heartbeats! [this storm-id])
  (teardown-topology-errors! [this storm-id])
  (heartbeat-storms [this])
  (error-topologies [this])

  (worker-heartbeat! [this storm-id node port info])
  (remove-worker-heartbeat! [this storm-id node port])
  (supervisor-heartbeat! [this supervisor-id info])
  (activate-storm! [this storm-id storm-base])
  (update-storm! [this storm-id new-elems])
  (remove-storm-base! [this storm-id])
  (set-assignment! [this storm-id info])
  (remove-storm! [this storm-id])
  (report-error [this storm-id task-id error])
  (errors [this storm-id task-id])

  (disconnect [this])
  )


(def ASSIGNMENTS-ROOT "assignments")
(def CODE-ROOT "code")
(def STORMS-ROOT "storms")
(def SUPERVISORS-ROOT "supervisors")
(def WORKERBEATS-ROOT "workerbeats")
(def ERRORS-ROOT "errors")

(def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
(def STORMS-SUBTREE (str "/" STORMS-ROOT))
(def SUPERVISORS-SUBTREE (str "/" SUPERVISORS-ROOT))
(def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
(def ERRORS-SUBTREE (str "/" ERRORS-ROOT))

(defn supervisor-path [id]
  (str SUPERVISORS-SUBTREE "/" id))

(defn assignment-path [id]
  (str ASSIGNMENTS-SUBTREE "/" id))

(defn storm-path [id]
  (str STORMS-SUBTREE "/" id))

(defn workerbeat-storm-root [storm-id]
  (str WORKERBEATS-SUBTREE "/" storm-id))

(defn workerbeat-path [storm-id node port]
  (str (workerbeat-storm-root storm-id) "/" node "-" port))

(defn error-storm-root [storm-id]
  (str ERRORS-SUBTREE "/" storm-id))

(defn error-path [storm-id component-id]
  (str (error-storm-root storm-id) "/" (url-encode component-id)))

(defn- issue-callback! [cb-atom]
  (let [cb @cb-atom]
    (reset! cb-atom nil)
    (when cb
      (cb))
    ))

(defn- issue-map-callback! [cb-atom id]
  (let [cb (@cb-atom id)]
    (swap! cb-atom dissoc id)
    (when cb
      (cb id))
    ))

(defn- maybe-deserialize [ser]
  (when ser
    (Utils/deserialize ser)))

(defstruct TaskError :error :time-secs)

(defn- parse-error-path [^String p]
  (Long/parseLong (.substring p 1)))


(defn convert-executor-beats [executors worker-hb]
  ;; ensures that we only return heartbeats for executors assigned to this worker
  (let [executor-stats (:executor-stats worker-hb)]
    (->> executors
      (map (fn [t] 
             (if (contains? executor-stats t)
               {t {:time-secs (:time-secs worker-hb)
                    :uptime (:uptime worker-hb)
                    :stats (get executor-stats t)}})))
      (into {}))))

;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
(defn mk-storm-cluster-state [cluster-state-spec]
  (let [[solo? cluster-state] (if (satisfies? ClusterState cluster-state-spec)
                                [false cluster-state-spec]
                                [true (mk-distributed-cluster-state cluster-state-spec)])
        assignment-info-callback (atom {})
        supervisors-callback (atom nil)
        assignments-callback (atom nil)
        storm-base-callback (atom {})
        state-id (register
                  cluster-state
                  (fn [type path]
                    (let [[subtree & args] (tokenize-path path)]
                      (condp = subtree
                          ASSIGNMENTS-ROOT (if (empty? args)
                                             (issue-callback! assignments-callback)
                                             (issue-map-callback! assignment-info-callback (first args)))
                          SUPERVISORS-ROOT (issue-callback! supervisors-callback)
                          STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
                          ;; this should never happen
                          (halt-process! 30 "Unknown callback for subtree " subtree args)
                          )
                      )))]
    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE]]
      (mkdirs cluster-state p))
    (reify
     StormClusterState
     
     (assignments [this callback]
        (when callback
          (reset! assignments-callback callback))
        (get-children cluster-state ASSIGNMENTS-SUBTREE (not-nil? callback)))
      
      (assignment-info [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)))
        )

      (active-storms [this]
        (get-children cluster-state STORMS-SUBTREE false)
        )

      (heartbeat-storms [this]
        (get-children cluster-state WORKERBEATS-SUBTREE false)
        )

      (error-topologies [this]
         (get-children cluster-state ERRORS-SUBTREE false)
        )

      (get-worker-heartbeat [this storm-id node port]
        (-> cluster-state
            (get-data (workerbeat-path storm-id node port) false)
            maybe-deserialize))

      (executor-beats [this storm-id executor->node+port]
        ;; need to take executor->node+port in explicitly so that we don't run into a situation where a 
        ;; long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
        ;; with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
        ;; we avoid situations like that
        (let [node+port->executors (reverse-map executor->node+port)
              all-heartbeats (for [[[node port] executors] node+port->executors]
                                (->> (get-worker-heartbeat this storm-id node port)
                                     (convert-executor-beats executors)
                                     ))]
          (apply merge all-heartbeats)))

      (supervisors [this callback]
        (when callback
          (reset! supervisors-callback callback))
        (get-children cluster-state SUPERVISORS-SUBTREE (not-nil? callback))
        )

      (supervisor-info [this supervisor-id]
        (maybe-deserialize (get-data cluster-state (supervisor-path supervisor-id) false))
        )

      (worker-heartbeat! [this storm-id node port info]
        (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize info)))

      (remove-worker-heartbeat! [this storm-id node port]
        (delete-node cluster-state (workerbeat-path storm-id node port))
        )

      (setup-heartbeats! [this storm-id]
        (mkdirs cluster-state (workerbeat-storm-root storm-id)))

      (teardown-heartbeats! [this storm-id]
        (try-cause
         (delete-node cluster-state (workerbeat-storm-root storm-id))
         (catch KeeperException e
           (log-warn-error e "Could not teardown heartbeats for " storm-id)
           )))

      (teardown-topology-errors! [this storm-id]
        (try-cause
         (delete-node cluster-state (error-storm-root storm-id))         
         (catch KeeperException e
           (log-warn-error e "Could not teardown errors for " storm-id)
           )))

      (supervisor-heartbeat! [this supervisor-id info]
        (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info))
        )

      (activate-storm! [this storm-id storm-base]
        (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base))
        )

      (update-storm! [this storm-id new-elems]
        (let [base (storm-base this storm-id nil)
              executors (:component->executors base)
              new-elems (update new-elems :component->executors (partial merge executors))]
          (set-data cluster-state (storm-path storm-id)
                                  (-> base
                                      (merge new-elems)
                                      Utils/serialize))))

      (storm-base [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)))
        )

      (remove-storm-base! [this storm-id]
        (delete-node cluster-state (storm-path storm-id))
        )

      (set-assignment! [this storm-id info]
        (set-data cluster-state (assignment-path storm-id) (Utils/serialize info))
        )

      (remove-storm! [this storm-id]
        (delete-node cluster-state (assignment-path storm-id))
        (remove-storm-base! this storm-id))

      (report-error [this storm-id component-id error]                
         (let [path (error-path storm-id component-id)
               data {:time-secs (current-time-secs) :error (stringify-error error)}
               _ (mkdirs cluster-state path)
               _ (create-sequential cluster-state (str path "/e") (Utils/serialize data))
               to-kill (->> (get-children cluster-state path false)
                            (sort-by parse-error-path)
                            reverse
                            (drop 10))]
           (doseq [k to-kill]
             (delete-node cluster-state (str path "/" k)))))

      (errors [this storm-id component-id]
         (let [path (error-path storm-id component-id)
               _ (mkdirs cluster-state path)
               children (get-children cluster-state path false)
               errors (dofor [c children]
                             (let [data (-> (get-data cluster-state (str path "/" c) false)
                                            maybe-deserialize)]
                               (when data
                                 (struct TaskError (:error data) (:time-secs data))
                                 )))
               ]
           (->> (filter not-nil? errors)
                (sort-by (comp - :time-secs)))))
      
      (disconnect [this]
        (unregister cluster-state state-id)
        (when solo?
          (close cluster-state)))
      )))

;; daemons have a single thread that will respond to events
;; start with initialize event
;; callbacks add events to the thread's queue

;; keeps in memory cache of the state, only for what client subscribes to. Any subscription is automatically kept in sync, and when there are changes, client is notified.
;; master gives orders through state, and client records status in state (ephemerally)

;; master tells nodes what workers to launch

;; master writes this. supervisors and workers subscribe to this to understand complete topology. each storm is a map from nodes to workers to tasks to ports whenever topology changes everyone will be notified
;; master includes timestamp of each assignment so that appropriate time can be given to each worker to start up
;; /assignments/{storm id}

;; which tasks they talk to, etc. (immutable until shutdown)
;; everyone reads this in full to understand structure
;; /tasks/{storm id}/{task id} ; just contains bolt id


;; supervisors send heartbeats here, master doesn't subscribe but checks asynchronously
;; /supervisors/status/{ephemeral node ids}  ;; node metadata such as port ranges are kept here 

;; tasks send heartbeats here, master doesn't subscribe, just checks asynchronously
;; /taskbeats/{storm id}/{ephemeral task id}

;; contains data about whether it's started or not, tasks and workers subscribe to specific storm here to know when to shutdown
;; master manipulates
;; /storms/{storm id}



;; Zookeeper flows:

;; Master:
;; job submit:
;; 1. read which nodes are available
;; 2. set up the worker/{storm}/{task} stuff (static)
;; 3. set assignments
;; 4. start storm - necessary in case master goes down, when goes back up can remember to take down the storm (2 states: on or off)

;; Monitoring (or by checking when nodes go down or heartbeats aren't received):
;; 1. read assignment
;; 2. see which tasks/nodes are up
;; 3. make new assignment to fix any problems
;; 4. if a storm exists but is not taken down fully, ensure that storm takedown is launched (step by step remove tasks and finally remove assignments)


;; masters only possible watches is on ephemeral nodes and tasks, and maybe not even

;; Supervisor:
;; 1. monitor /storms/* and assignments
;; 2. local state about which workers are local
;; 3. when storm is on, check that workers are running locally & start/kill if different than assignments
;; 4. when storm is off, monitor tasks for workers - when they all die or don't hearbeat, kill the process and cleanup

;; Worker:
;; 1. On startup, start the tasks if the storm is on

;; Task:
;; 1. monitor assignments, reroute when assignments change
;; 2. monitor storm (when storm turns off, error if assignments change) - take down tasks as master turns them off



;; locally on supervisor: workers write pids locally on startup, supervisor deletes it on shutdown (associates pid with worker name)
;; supervisor periodically checks to make sure processes are alive
;; {rootdir}/workers/{storm id}/{worker id}   ;; contains pid inside

;; all tasks in a worker share the same cluster state
;; workers, supervisors, and tasks subscribes to storm to know when it's started or stopped
;; on stopped, master removes records in order (tasks need to subscribe to themselves to see if they disappear)
;; when a master removes a worker, the supervisor should kill it (and escalate to kill -9)
;; on shutdown, tasks subscribe to tasks that send data to them to wait for them to die. when node disappears, they can die


================================================
FILE: storm-core/src/clj/backtype/storm/command/activate.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.activate
  (:use [backtype.storm thrift log])
  (:gen-class))

(defn -main [name] 
  (with-configured-nimbus-connection nimbus
    (.activate nimbus name)
    (log-message "Activated topology: " name)
    ))


================================================
FILE: storm-core/src/clj/backtype/storm/command/config_value.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.config-value
  (:use [backtype.storm config log])
  (:gen-class))


(defn -main [^String name]
  (let [conf (read-storm-config)]
    (println "VALUE:" (conf name))
    ))


================================================
FILE: storm-core/src/clj/backtype/storm/command/deactivate.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.deactivate
  (:use [backtype.storm thrift log])
  (:gen-class))

(defn -main [name] 
  (with-configured-nimbus-connection nimbus
    (.deactivate nimbus name)
    (log-message "Deactivated topology: " name)
    ))


================================================
FILE: storm-core/src/clj/backtype/storm/command/dev_zookeeper.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.dev-zookeeper
  (:use [backtype.storm zookeeper util config])
  (:gen-class))

(defn -main [& args]
  (let [conf (read-storm-config)
        port (conf STORM-ZOOKEEPER-PORT)
        localpath (conf DEV-ZOOKEEPER-PATH)]
    (rmr localpath)
    (mk-inprocess-zookeeper localpath :port port)
    ))


================================================
FILE: storm-core/src/clj/backtype/storm/command/kill_topology.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.kill-topology
  (:use [clojure.tools.cli :only [cli]])
  (:use [backtype.storm thrift config log])
  (:import [backtype.storm.generated KillOptions])
  (:gen-class))

(defn -main [& args]
  (let [[{wait :wait} [name] _] (cli args ["-w" "--wait" :default nil :parse-fn #(Integer/parseInt %)])
        opts (KillOptions.)]
    (if wait (.set_wait_secs opts wait))
    (with-configured-nimbus-connection nimbus
      (.killTopologyWithOpts nimbus name opts)
      (log-message "Killed topology: " name)
      )))


================================================
FILE: storm-core/src/clj/backtype/storm/command/list.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.list
  (:use [backtype.storm thrift log])
  (:import [backtype.storm.generated TopologySummary])
  (:gen-class))

(defn -main []
  (with-configured-nimbus-connection nimbus
    (let [cluster-info (.getClusterInfo nimbus)
          topologies (.get_topologies cluster-info)
          msg-format "%-20s %-10s %-10s %-12s %-10s"]
      (if (or (nil? topologies) (empty? topologies))
        (println "No topologies running.")
        (do
          (println (format msg-format "Topology_name" "Status" "Num_tasks" "Num_workers" "Uptime_secs"))
          (println "-------------------------------------------------------------------")
          (doseq [^TopologySummary topology topologies]
            (let [topology-name (.get_name topology)
                  topology-status (.get_status topology)
                  topology-num-tasks (.get_num_tasks topology)
                  topology-num-workers (.get_num_workers topology)
                  topology-uptime-secs (.get_uptime_secs topology)]
              (println (format msg-format  topology-name topology-status topology-num-tasks
                               topology-num-workers topology-uptime-secs)))))))))

================================================
FILE: storm-core/src/clj/backtype/storm/command/rebalance.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.rebalance
  (:use [clojure.tools.cli :only [cli]])
  (:use [backtype.storm thrift config log])
  (:import [backtype.storm.generated RebalanceOptions])
  (:gen-class))

(defn- parse-executor [^String s]
  (let [eq-pos (.lastIndexOf s "=")
        name (.substring s 0 eq-pos)
        amt (.substring s (inc eq-pos))]
    {name (Integer/parseInt amt)}
    ))

(defn -main [& args] 
  (let [[{wait :wait executor :executor num-workers :num-workers} [name] _]
                  (cli args ["-w" "--wait" :default nil :parse-fn #(Integer/parseInt %)]
                            ["-n" "--num-workers" :default nil :parse-fn #(Integer/parseInt %)]
                            ["-e" "--executor" :combine-fn merge :parse-fn parse-executor])
        opts (RebalanceOptions.)]
    (if wait (.set_wait_secs opts wait))
    (if executor (.set_num_executors opts executor))
    (if num-workers (.set_num_workers opts num-workers))
    (with-configured-nimbus-connection nimbus
      (.rebalance nimbus name opts)
      (log-message "Topology " name " is rebalancing")
      )))


================================================
FILE: storm-core/src/clj/backtype/storm/command/shell_submission.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.command.shell-submission
  (:import [backtype.storm StormSubmitter])
  (:use [backtype.storm thrift util config log])
  (:require [clojure.string :as str])
  (:gen-class))


(defn -main [^String tmpjarpath & args]
  (let [conf (read-storm-config)
        host (conf NIMBUS-HOST)
        port (conf NIMBUS-THRIFT-PORT)
        jarpath (StormSubmitter/submitJar conf tmpjarpath)
        args (concat args [host port jarpath])]
    (exec-command! (str/join " " args))
    ))


================================================
FILE: storm-core/src/clj/backtype/storm/config.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.config
  (:import [java.io FileReader File])
  (:import [backtype.storm Config ConfigValidation$FieldValidator])
  (:import [backtype.storm.utils Utils LocalState])
  (:import [org.apache.commons.io FileUtils])
  (:require [clojure [string :as str]])
  (:use [backtype.storm util])
  )

(def RESOURCES-SUBDIR "resources")

(defn- clojure-config-name [name]
  (.replace (.toUpperCase name) "_" "-"))

;; define clojure constants for every configuration parameter
(doseq [f (seq (.getFields Config))]
  (let [name (.getName f)
        new-name (clojure-config-name name)]
    (eval
      `(def ~(symbol new-name) (. Config ~(symbol name))))
      ))

(def ALL-CONFIGS
  (dofor [f (seq (.getFields Config))]
         (.get f nil)
         ))

(defmulti get-FieldValidator class-selector)

(defmethod get-FieldValidator nil [_]
  (throw (IllegalArgumentException. "Cannot validate a nil field.")))

(defmethod get-FieldValidator
  ConfigValidation$FieldValidator [validator] validator)

(defmethod get-FieldValidator Object [klass]
  {:pre [(not (nil? klass))]}
  (reify ConfigValidation$FieldValidator
    (validateField [this name v]
      (if (and (not (nil? v))
               (not (instance? klass v)))
        (throw (IllegalArgumentException.
                 (str "field " name " '" v "' must be a '" (.getName klass) "'")))))))

;; Create a mapping of config-string -> validator
;; Config fields must have a _SCHEMA field defined
(def CONFIG-SCHEMA-MAP
  (->> (.getFields Config)
          (filter #(not (re-matches #".*_SCHEMA$" (.getName %))))
          (map (fn [f] [(.get f nil) (get-FieldValidator
                                       (-> Config
                                         (.getField (str (.getName f) "_SCHEMA"))
                                         (.get nil)))]))
          (into {})))

(defn cluster-mode [conf & args]
  (keyword (conf STORM-CLUSTER-MODE)))

(defn local-mode? [conf]
  (let [mode (conf STORM-CLUSTER-MODE)]
    (condp = mode
      "local" true
      "distributed" false
      (throw (IllegalArgumentException.
                (str "Illegal cluster mode in conf: " mode)))
      )))

(defn sampling-rate [conf]
  (->> (conf TOPOLOGY-STATS-SAMPLE-RATE)
       (/ 1)
       int))

(defn mk-stats-sampler [conf]
  (even-sampler (sampling-rate conf)))

; storm.zookeeper.servers:
;     - "server1"
;     - "server2"
;     - "server3"
; nimbus.host: "master"
; 
; ########### These all have default values as shown
; 
; ### storm.* configs are general configurations
; # the local dir is where jars are kept
; storm.local.dir: "/mnt/storm"
; storm.zookeeper.port: 2181
; storm.zookeeper.root: "/storm"

(defn read-default-config []
  (clojurify-structure (Utils/readDefaultConfig)))

(defn- validate-configs-with-schemas [conf]
  (doseq [[k v] conf
         :let [schema (CONFIG-SCHEMA-MAP k)]]
    (if (not (nil? schema))
      (.validateField schema k v))))

(defn read-storm-config []
  (let [
        conf (clojurify-structure (Utils/readStormConfig))]
    (validate-configs-with-schemas conf)
    conf))

(defn read-yaml-config [name]
  (let [conf (clojurify-structure (Utils/findAndReadConfigFile name true))]
    (validate-configs-with-schemas conf)
    conf))

(defn master-local-dir [conf]
  (let [ret (str (conf STORM-LOCAL-DIR) "/nimbus")]
    (FileUtils/forceMkdir (File. ret))
    ret
    ))

(defn master-stormdist-root
  ([conf]
     (str (master-local-dir conf) "/stormdist"))
  ([conf storm-id]
     (str (master-stormdist-root conf) "/" storm-id)))

(defn master-stormjar-path [stormroot]
  (str stormroot "/stormjar.jar"))

(defn master-stormcode-path [stormroot]
  (str stormroot "/stormcode.ser"))

(defn master-stormconf-path [stormroot]
  (str stormroot "/stormconf.ser"))

(defn master-inbox [conf]
  (let [ret (str (master-local-dir conf) "/inbox")]
    (FileUtils/forceMkdir (File. ret))
    ret ))

(defn master-inimbus-dir [conf]
  (str (master-local-dir conf) "/inimbus"))

(defn supervisor-local-dir [conf]
  (let [ret (str (conf STORM-LOCAL-DIR) "/supervisor")]
    (FileUtils/forceMkdir (File. ret))
    ret
    ))

(defn supervisor-isupervisor-dir [conf]
  (str (supervisor-local-dir conf) "/isupervisor"))

(defn supervisor-stormdist-root
  ([conf] (str (supervisor-local-dir conf) "/stormdist"))
  ([conf storm-id]
      (str (supervisor-stormdist-root conf) "/" (java.net.URLEncoder/encode storm-id))))

(defn supervisor-stormjar-path [stormroot]
  (str stormroot "/stormjar.jar"))

(defn supervisor-stormcode-path [stormroot]
  (str stormroot "/stormcode.ser"))

(defn supervisor-stormconf-path [stormroot]
  (str stormroot "/stormconf.ser"))

(defn supervisor-tmp-dir [conf]
  (let [ret (str (supervisor-local-dir conf) "/tmp")]
    (FileUtils/forceMkdir (File. ret))
    ret ))

(defn supervisor-storm-resources-path [stormroot]
  (str stormroot "/" RESOURCES-SUBDIR))

(defn ^LocalState supervisor-state [conf]
  (LocalState. (str (supervisor-local-dir conf) "/localstate")))

(defn read-supervisor-storm-conf [conf storm-id]
  (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))))
    ))

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

(defn worker-root
  ([conf]
     (str (conf STORM-LOCAL-DIR) "/workers"))
  ([conf id]
     (str (worker-root conf) "/" id)))

(defn worker-pids-root
  [conf id]
  (str (worker-root conf id) "/pids"))

(defn worker-pid-path [conf id pid]
  (str (worker-pids-root conf id) "/" pid))

(defn worker-heartbeats-root
  [conf id]
  (str (worker-root conf id) "/heartbeats"))

;; workers heartbeat here with pid and timestamp
;; if supervisor stops receiving heartbeat, it kills and restarts the process
;; in local mode, keep a global map of ids to threads for simulating process management
(defn ^LocalState worker-state  [conf id]
  (LocalState. (worker-heartbeats-root conf id)))


================================================
FILE: storm-core/src/clj/backtype/storm/daemon/acker.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.acker
  (:import [backtype.storm.task OutputCollector TopologyContext IBolt])
  (:import [backtype.storm.tuple Tuple Fields])
  (:import [backtype.storm.utils RotatingMap MutableObject])
  (:import [java.util List Map])
  (:import [backtype.storm Constants])
  (:use [backtype.storm config util log])
  (:gen-class
   :init init
   :implements [backtype.storm.task.IBolt]
   :constructors {[] []}
   :state state ))

(def ACKER-COMPONENT-ID "__acker")
(def ACKER-INIT-STREAM-ID "__ack_init")
(def ACKER-ACK-STREAM-ID "__ack_ack")
(def ACKER-FAIL-STREAM-ID "__ack_fail")

(defn- update-ack [curr-entry val]
  (let [old (get curr-entry :val 0)]
    (assoc curr-entry :val (bit-xor old val))
    ))

(defn- acker-emit-direct [^OutputCollector collector ^Integer task ^String stream ^List values]
  (.emitDirect collector task stream values)
  )

(defn mk-acker-bolt []
  (let [output-collector (MutableObject.)
        pending (MutableObject.)]
    (reify IBolt
      (^void prepare [this ^Map storm-conf ^TopologyContext context ^OutputCollector collector]
               (.setObject output-collector collector)
               (.setObject pending (RotatingMap. 2))
               )
      (^void execute [this ^Tuple tuple]
             (let [^RotatingMap pending (.getObject pending)
                   stream-id (.getSourceStreamId tuple)]
               (if (= stream-id Constants/SYSTEM_TICK_STREAM_ID)
                 (.rotate pending)
                 (let [id (.getValue tuple 0)
                       ^OutputCollector output-collector (.getObject output-collector)
                       curr (.get pending id)
                       curr (condp = stream-id
                                ACKER-INIT-STREAM-ID (-> curr
                                                         (update-ack (.getValue tuple 1))
                                                         (assoc :spout-task (.getValue tuple 2)))
                                ACKER-ACK-STREAM-ID (update-ack curr (.getValue tuple 1))
                                ACKER-FAIL-STREAM-ID (assoc curr :failed true))]
                   (.put pending id curr)
                   (when (and curr (:spout-task curr))
                     (cond (= 0 (:val curr))
                           (do
                             (.remove pending id)
                             (acker-emit-direct output-collector
                                                (:spout-task curr)
                                                ACKER-ACK-STREAM-ID
                                                [id]
                                                ))
                           (:failed curr)
                           (do
                             (.remove pending id)
                             (acker-emit-direct output-collector
                                                (:spout-task curr)
                                                ACKER-FAIL-STREAM-ID
                                                [id]
                                                ))
                           ))
                   (.ack output-collector tuple)
                   ))))
      (^void cleanup [this]
        )
      )))

(defn -init []
  [[] (container)])

(defn -prepare [this conf context collector]
  (let [^IBolt ret (mk-acker-bolt)]
    (container-set! (.state ^backtype.storm.daemon.acker this) ret)
    (.prepare ret conf context collector)
    ))

(defn -execute [this tuple]
  (let [^IBolt delegate (container-get (.state ^backtype.storm.daemon.acker this))]
    (.execute delegate tuple)
    ))

(defn -cleanup [this]
  (let [^IBolt delegate (container-get (.state ^backtype.storm.daemon.acker this))]
    (.cleanup delegate)
    ))


================================================
FILE: storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.builtin-metrics
  (:import [backtype.storm.metric.api MultiCountMetric MultiReducedMetric MeanReducer StateMetric])
  (:import [backtype.storm Config])
  (:use [backtype.storm.stats :only [stats-rate]]))

(defrecord BuiltinSpoutMetrics [^MultiCountMetric ack-count                                
                                ^MultiReducedMetric complete-latency
                                ^MultiCountMetric fail-count
                                ^MultiCountMetric emit-count
                                ^MultiCountMetric transfer-count])
(defrecord BuiltinBoltMetrics [^MultiCountMetric ack-count
                               ^MultiReducedMetric process-latency
                               ^MultiCountMetric fail-count
                               ^MultiCountMetric execute-count
                               ^MultiReducedMetric execute-latency
                               ^MultiCountMetric emit-count
                               ^MultiCountMetric transfer-count])

(defn make-data [executor-type]
  (condp = executor-type
    :spout (BuiltinSpoutMetrics. (MultiCountMetric.)
                                 (MultiReducedMetric. (MeanReducer.))
                                 (MultiCountMetric.)
                                 (MultiCountMetric.)
                                 (MultiCountMetric.))
    :bolt (BuiltinBoltMetrics. (MultiCountMetric.)
                               (MultiReducedMetric. (MeanReducer.))
                               (MultiCountMetric.)
                               (MultiCountMetric.)
                               (MultiReducedMetric. (MeanReducer.))
                               (MultiCountMetric.)
                               (MultiCountMetric.))))

(defn register-all [builtin-metrics  storm-conf topology-context]
  (doseq [[kw imetric] builtin-metrics]
    (.registerMetric topology-context (str "__" (name kw)) imetric
                     (int (get storm-conf Config/TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)))))
          
(defn register-queue-metrics [queues storm-conf topology-context]
  (doseq [[qname q] queues]
    (.registerMetric topology-context (str "__" (name qname)) (StateMetric. q)
                     (int (get storm-conf Config/TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)))))

(defn spout-acked-tuple! [^BuiltinSpoutMetrics m stats stream latency-ms]  
  (-> m .ack-count (.scope stream) (.incrBy (stats-rate stats)))
  (-> m .complete-latency (.scope stream) (.update latency-ms)))

(defn spout-failed-tuple! [^BuiltinSpoutMetrics m stats stream]  
  (-> m .fail-count (.scope stream) (.incrBy (stats-rate stats))))

(defn bolt-execute-tuple! [^BuiltinBoltMetrics m stats comp-id stream latency-ms]
  (let [scope (str comp-id ":" stream)]    
    (-> m .execute-count (.scope scope) (.incrBy (stats-rate stats)))
    (-> m .execute-latency (.scope scope) (.update latency-ms))))

(defn bolt-acked-tuple! [^BuiltinBoltMetrics m stats comp-id stream latency-ms]
  (let [scope (str comp-id ":" stream)]
    (-> m .ack-count (.scope scope) (.incrBy (stats-rate stats)))
    (-> m .process-latency (.scope scope) (.update latency-ms))))

(defn bolt-failed-tuple! [^BuiltinBoltMetrics m stats comp-id stream]
  (let [scope (str comp-id ":" stream)]    
    (-> m .fail-count (.scope scope) (.incrBy (stats-rate stats)))))

(defn emitted-tuple! [m stats stream]
  (-> m :emit-count (.scope stream) (.incrBy (stats-rate stats))))

(defn transferred-tuple! [m stats stream num-out-tasks]
  (-> m :transfer-count (.scope stream) (.incrBy (* num-out-tasks (stats-rate stats)))))


================================================
FILE: storm-core/src/clj/backtype/storm/daemon/common.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.common
  (:use [backtype.storm log config util])
  (:import [backtype.storm.generated StormTopology
            InvalidTopologyException GlobalStreamId])
  (:import [backtype.storm.utils Utils])
  (:import [backtype.storm.task WorkerTopologyContext])
  (:import [backtype.storm Constants])
  (:import [backtype.storm.metric SystemBolt])
  (:require [clojure.set :as set])  
  (:require [backtype.storm.daemon.acker :as acker])
  (:require [backtype.storm.thrift :as thrift])
  )

(defn system-id? [id]
  (Utils/isSystemId id))

(def ACKER-COMPONENT-ID acker/ACKER-COMPONENT-ID)
(def ACKER-INIT-STREAM-ID acker/ACKER-INIT-STREAM-ID)
(def ACKER-ACK-STREAM-ID acker/ACKER-ACK-STREAM-ID)
(def ACKER-FAIL-STREAM-ID acker/ACKER-FAIL-STREAM-ID)

(def SYSTEM-STREAM-ID "__system")

(def SYSTEM-COMPONENT-ID Constants/SYSTEM_COMPONENT_ID)
(def SYSTEM-TICK-STREAM-ID Constants/SYSTEM_TICK_STREAM_ID)
(def METRICS-STREAM-ID Constants/METRICS_STREAM_ID)
(def METRICS-TICK-STREAM-ID Constants/METRICS_TICK_STREAM_ID)

;; the task id is the virtual port
;; node->host is here so that tasks know who to talk to just from assignment
;; this avoid situation where node goes down and task doesn't know what to do information-wise
(defrecord Assignment [master-code-dir node->host executor->node+port executor->start-time-secs])


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

(defrecord SupervisorInfo [time-secs hostname assignment-id used-ports meta scheduler-meta uptime-secs])

(defprotocol DaemonCommon
  (waiting? [this]))

(def LS-WORKER-HEARTBEAT "worker-heartbeat")

;; LocalState constants
(def LS-ID "supervisor-id")
(def LS-LOCAL-ASSIGNMENTS "local-assignments")
(def LS-APPROVED-WORKERS "approved-workers")



(defrecord WorkerHeartbeat [time-secs storm-id executors port])

(defrecord ExecutorStats [^long processed
                          ^long acked
                          ^long emitted
                          ^long transferred
                          ^long failed])

(defn new-executor-stats []
  (ExecutorStats. 0 0 0 0 0))

(defn get-storm-id [storm-cluster-state storm-name]
  (let [active-storms (.active-storms storm-cluster-state)]
    (find-first
      #(= storm-name (:storm-name (.storm-base storm-cluster-state % nil)))
      active-storms)
    ))

(defn topology-bases [storm-cluster-state]
  (let [active-topologies (.active-storms storm-cluster-state)]
    (into {} 
          (dofor [id active-topologies]
                 [id (.storm-base storm-cluster-state id nil)]
                 ))
    ))

(defn validate-distributed-mode! [conf]
  (if (local-mode? conf)
      (throw
        (IllegalArgumentException. "Cannot start server in local mode!"))))

(defmacro defserverfn [name & body]
  `(let [exec-fn# (fn ~@body)]
    (defn ~name [& args#]
      (try-cause
        (apply exec-fn# args#)
      (catch InterruptedException e#
        (throw e#))
      (catch Throwable t#
        (log-error t# "Error on initialization of server " ~(str name))
        (halt-process! 13 "Error on initialization")
        )))))

(defn- validate-ids! [^StormTopology topology]
  (let [sets (map #(.getFieldValue topology %) thrift/STORM-TOPOLOGY-FIELDS)
        offending (apply any-intersection sets)]
    (if-not (empty? offending)
      (throw (InvalidTopologyException.
              (str "Duplicate component ids: " offending))))
    (doseq [f thrift/STORM-TOPOLOGY-FIELDS
            :let [obj-map (.getFieldValue topology f)]]
      (doseq [id (keys obj-map)]
        (if (system-id? id)
          (throw (InvalidTopologyException.
                  (str id " is not a valid component id")))))
      (doseq [obj (vals obj-map)
              id (-> obj .get_common .get_streams keys)]
        (if (system-id? id)
          (throw (InvalidTopologyException.
                  (str id " is not a valid stream id"))))))
    ))

(defn all-components [^StormTopology topology]
  (apply merge {}
         (for [f thrift/STORM-TOPOLOGY-FIELDS]
           (.getFieldValue topology f)
           )))

(defn component-conf [component]
  (->> component
      .get_common
      .get_json_conf
      from-json))

(defn validate-basic! [^StormTopology topology]
  (validate-ids! topology)
  (doseq [f thrift/SPOUT-FIELDS
          obj (->> f (.getFieldValue topology) vals)]
    (if-not (empty? (-> obj .get_common .get_inputs))
      (throw (InvalidTopologyException. "May not declare inputs for a spout"))))
  (doseq [[comp-id comp] (all-components topology)
          :let [conf (component-conf comp)
                p (-> comp .get_common thrift/parallelism-hint)]]
    (when (and (> (conf TOPOLOGY-TASKS) 0)
               p
               (<= p 0))
      (throw (InvalidTopologyException. "Number of executors must be greater than 0 when number of tasks is greater than 0"))
      )))

(defn validate-structure! [^StormTopology topology]
  ;; validate all the component subscribe from component+stream which actually exists in the topology
  ;; and if it is a fields grouping, validate the corresponding field exists  
  (let [all-components (all-components topology)]
    (doseq [[id comp] all-components
            :let [inputs (.. comp get_common get_inputs)]]
      (doseq [[global-stream-id grouping] inputs
              :let [source-component-id (.get_componentId global-stream-id)
                    source-stream-id    (.get_streamId global-stream-id)]]
        (if-not (contains? all-components source-component-id)
          (throw (InvalidTopologyException. (str "Component: [" id "] subscribes from non-existent component [" source-component-id "]")))
          (let [source-streams (-> all-components (get source-component-id) .get_common .get_streams)]
            (if-not (contains? source-streams source-stream-id)
              (throw (InvalidTopologyException. (str "Component: [" id "] subscribes from non-existent stream: [" source-stream-id "] of component [" source-component-id "]")))
              (if (= :fields (thrift/grouping-type grouping))
                (let [grouping-fields (set (.get_fields grouping))
                      source-stream-fields (-> source-streams (get source-stream-id) .get_output_fields set)
                      diff-fields (set/difference grouping-fields source-stream-fields)]
                  (when-not (empty? diff-fields)
                    (throw (InvalidTopologyException. (str "Component: [" id "] subscribes from stream: [" source-stream-id "] of component [" source-component-id "] with non-existent fields: " diff-fields)))))))))))))

(defn acker-inputs [^StormTopology topology]
  (let [bolt-ids (.. topology get_bolts keySet)
        spout-ids (.. topology get_spouts keySet)
        spout-inputs (apply merge
                            (for [id spout-ids]
                              {[id ACKER-INIT-STREAM-ID] ["id"]}
                              ))
        bolt-inputs (apply merge
                           (for [id bolt-ids]
                             {[id ACKER-ACK-STREAM-ID] ["id"]
                              [id ACKER-FAIL-STREAM-ID] ["id"]}
                             ))]
    (merge spout-inputs bolt-inputs)))

(defn add-acker! [storm-conf ^StormTopology ret]
  (let [num-executors (if (nil? (storm-conf TOPOLOGY-ACKER-EXECUTORS)) (storm-conf TOPOLOGY-WORKERS) (storm-conf TOPOLOGY-ACKER-EXECUTORS))
        acker-bolt (thrift/mk-bolt-spec* (acker-inputs ret)
                                         (new backtype.storm.daemon.acker)
                                         {ACKER-ACK-STREAM-ID (thrift/direct-output-fields ["id"])
                                          ACKER-FAIL-STREAM-ID (thrift/direct-output-fields ["id"])
                                          }
                                         :p num-executors
                                         :conf {TOPOLOGY-TASKS num-executors
                                                TOPOLOGY-TICK-TUPLE-FREQ-SECS (storm-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)})]
    (dofor [[_ bolt] (.get_bolts ret)
            :let [common (.get_common bolt)]]
           (do
             (.put_to_streams common ACKER-ACK-STREAM-ID (thrift/output-fields ["id" "ack-val"]))
             (.put_to_streams common ACKER-FAIL-STREAM-ID (thrift/output-fields ["id"]))
             ))
    (dofor [[_ spout] (.get_spouts ret)
            :let [common (.get_common spout)
                  spout-conf (merge
                               (component-conf spout)
                               {TOPOLOGY-TICK-TUPLE-FREQ-SECS (storm-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)})]]
      (do
        ;; this set up tick tuples to cause timeouts to be triggered
        (.set_json_conf common (to-json spout-conf))
        (.put_to_streams common ACKER-INIT-STREAM-ID (thrift/output-fields ["id" "init-val" "spout-task"]))
        (.put_to_inputs common
                        (GlobalStreamId. ACKER-COMPONENT-ID ACKER-ACK-STREAM-ID)
                        (thrift/mk-direct-grouping))
        (.put_to_inputs common
                        (GlobalStreamId. ACKER-COMPONENT-ID ACKER-FAIL-STREAM-ID)
                        (thrift/mk-direct-grouping))
        ))
    (.put_to_bolts ret "__acker" acker-bolt)
    ))

(defn add-metric-streams! [^StormTopology topology]
  (doseq [[_ component] (all-components topology)
          :let [common (.get_common component)]]
    (.put_to_streams common METRICS-STREAM-ID
                     (thrift/output-fields ["task-info" "data-points"]))))

(defn add-system-streams! [^StormTopology topology]
  (doseq [[_ component] (all-components topology)
          :let [common (.get_common component)]]
    (.put_to_streams common SYSTEM-STREAM-ID (thrift/output-fields ["event"]))))


(defn map-occurrences [afn coll]
  (->> coll
       (reduce (fn [[counts new-coll] x]
                 (let [occurs (inc (get counts x 0))]
                   [(assoc counts x occurs) (cons (afn x occurs) new-coll)]))
               [{} []])
       (second)
       (reverse)))

(defn number-duplicates [coll]
  "(number-duplicates [\"a\", \"b\", \"a\"]) => [\"a\", \"b\", \"a#2\"]"
  (map-occurrences (fn [x occurences] (if (>= occurences 2) (str x "#" occurences) x)) coll))

(defn metrics-consumer-register-ids [storm-conf]
  "Generates a list of component ids for each metrics consumer
   e.g. [\"__metrics_org.mycompany.MyMetricsConsumer\", ..] "
  (->> (get storm-conf TOPOLOGY-METRICS-CONSUMER-REGISTER)         
       (map #(get % "class"))
       (number-duplicates)
       (map #(str Constants/METRICS_COMPONENT_ID_PREFIX %))))

(defn metrics-consumer-bolt-specs [storm-conf topology]
  (let [component-ids-that-emit-metrics (cons SYSTEM-COMPONENT-ID (keys (all-components topology)))
        inputs (->> (for [comp-id component-ids-that-emit-metrics]
                      {[comp-id METRICS-STREAM-ID] :shuffle})
                    (into {}))
        
        mk-bolt-spec (fn [class arg p]
                       (thrift/mk-bolt-spec*
                        inputs
                        (backtype.storm.metric.MetricsConsumerBolt. class arg)
                        {} :p p :conf {TOPOLOGY-TASKS p}))]
    
    (map
     (fn [component-id register]           
       [component-id (mk-bolt-spec (get register "class")
                                   (get register "argument")
                                   (or (get register "parallelism.hint") 1))])
     
     (metrics-consumer-register-ids storm-conf)
     (get storm-conf TOPOLOGY-METRICS-CONSUMER-REGISTER))))

(defn add-metric-components! [storm-conf ^StormTopology topology]  
  (doseq [[comp-id bolt-spec] (metrics-consumer-bolt-specs storm-conf topology)]
    (.put_to_bolts topology comp-id bolt-spec)))

(defn add-system-components! [conf ^StormTopology topology]
  (let [system-bolt-spec (thrift/mk-bolt-spec*
                          {}
                          (SystemBolt.)
                          {SYSTEM-TICK-STREAM-ID (thrift/output-fields ["rate_secs"])
                           METRICS-TICK-STREAM-ID (thrift/output-fields ["interval"])}                          
                          :p 0
                          :conf {TOPOLOGY-TASKS 0})]
    (.put_to_bolts topology SYSTEM-COMPONENT-ID system-bolt-spec)))

(defn system-topology! [storm-conf ^StormTopology topology]
  (validate-basic! topology)
  (let [ret (.deepCopy topology)]
    (add-acker! storm-conf ret)
    (add-metric-components! storm-conf ret)    
    (add-system-components! storm-conf ret)
    (add-metric-streams! ret)
    (add-system-streams! ret)
    (validate-structure! ret)
    ret
    ))

(defn has-ackers? [storm-conf]
  (or (nil? (storm-conf TOPOLOGY-ACKER-EXECUTORS)) (> (storm-conf TOPOLOGY-ACKER-EXECUTORS) 0)))


(defn num-start-executors [component]
  (thrift/parallelism-hint (.get_common component)))

(defn storm-task-info
  "Returns map from task -> component id"
  [^StormTopology user-topology storm-conf]
  (->> (system-topology! storm-conf user-topology)
       all-components
       (map-val (comp #(get % TOPOLOGY-TASKS) component-conf))
       (sort-by first)
       (mapcat (fn [[c num-tasks]] (repeat num-tasks c)))
       (map (fn [id comp] [id comp]) (iterate (comp int inc) (int 1)))
       (into {})
       ))

(defn executor-id->tasks [[first-task-id last-task-id]]
  (->> (range first-task-id (inc last-task-id))
       (map int)))

(defn worker-context [worker]
  (WorkerTopologyContext. (:system-topology worker)
                          (:storm-conf worker)
                          (:task->component worker)
                          (:component->sorted-tasks worker)
                          (:component->stream->fields worker)
                          (:storm-id worker)
                          (supervisor-storm-resources-path
                            (supervisor-stormdist-root (:conf worker) (:storm-id worker)))
                          (worker-pids-root (:conf worker) (:worker-id worker))
                          (:port worker)
                          (:task-ids worker)
                          (:default-shared-resources worker)
                          (:user-shared-resources worker)
                          ))


(defn to-task->node+port [executor->node+port]
  (->> executor->node+port
       (mapcat (fn [[e node+port]] (for [t (executor-id->tasks e)] [t node+port])))
       (into {})))


================================================
FILE: storm-core/src/clj/backtype/storm/daemon/drpc.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.drpc
  (:import [org.apache.thrift7.server THsHaServer THsHaServer$Args])
  (:import [org.apache.thrift7.protocol TBinaryProtocol TBinaryProtocol$Factory])
  (:import [org.apache.thrift7 TException])
  (:import [org.apache.thrift7.transport TNonblockingServerTransport TNonblockingServerSocket])
  (:import [backtype.storm.generated DistributedRPC DistributedRPC$Iface DistributedRPC$Processor
            DRPCRequest DRPCExecutionException DistributedRPCInvocations DistributedRPCInvocations$Iface
            DistributedRPCInvocations$Processor])
  (:import [java.util.concurrent Semaphore ConcurrentLinkedQueue ThreadPoolExecutor ArrayBlockingQueue TimeUnit])
  (:import [backtype.storm.daemon Shutdownable])
  (:import [java.net InetAddress])
  (:use [backtype.storm bootstrap config log])
  (:gen-class))

(bootstrap)

(def TIMEOUT-CHECK-SECS 5)

(defn acquire-queue [queues-atom function]
  (swap! queues-atom
    (fn [amap]
      (if-not (amap function)
        (assoc amap function (ConcurrentLinkedQueue.))
        amap)
        ))
  (@queues-atom function))

;; TODO: change this to use TimeCacheMap
(defn service-handler []
  (let [conf (read-storm-config)
        ctr (atom 0)
        id->sem (atom {})
        id->result (atom {})
        id->start (atom {})
        request-queues (atom {})
        cleanup (fn [id] (swap! id->sem dissoc id)
                         (swap! id->result dissoc id)
                         (swap! id->start dissoc id))
        my-ip (.getHostAddress (InetAddress/getLocalHost))
        clear-thread (async-loop
                      (fn []
                        (doseq [[id start] @id->start]
                          (when (> (time-delta start) (conf DRPC-REQUEST-TIMEOUT-SECS))
                            (when-let [sem (@id->sem id)]
                              (swap! id->result assoc id (DRPCExecutionException. "Request timed out"))
                              (.release sem))
                            (cleanup id)
                            ))
                        TIMEOUT-CHECK-SECS
                        ))
        ]
    (reify DistributedRPC$Iface
      (^String execute [this ^String function ^String args]
        (log-debug "Received DRPC request for " function " " args " at " (System/currentTimeMillis))
        (let [id (str (swap! ctr (fn [v] (mod (inc v) 1000000000))))
              ^Semaphore sem (Semaphore. 0)
              req (DRPCRequest. args id)
              ^ConcurrentLinkedQueue queue (acquire-queue request-queues function)
              ]
          (swap! id->start assoc id (current-time-secs))
          (swap! id->sem assoc id sem)
          (.add queue req)
          (log-debug "Waiting for DRPC result for " function " " args " at " (System/currentTimeMillis))
          (.acquire sem)
          (log-debug "Acquired DRPC result for " function " " args " at " (System/currentTimeMillis))
          (let [result (@id->result id)]
            (cleanup id)
            (log-debug "Returning DRPC result for " function " " args " at " (System/currentTimeMillis))
            (if (instance? DRPCExecutionException result)
              (throw result)
              result
              ))))
      DistributedRPCInvocations$Iface
      (^void result [this ^String id ^String result]
        (let [^Semaphore sem (@id->sem id)]
          (log-debug "Received result " result " for " id " at " (System/currentTimeMillis))
          (when sem
            (swap! id->result assoc id result)
            (.release sem)
            )))
      (^void failRequest [this ^String id]
        (let [^Semaphore sem (@id->sem id)]
          (when sem
            (swap! id->result assoc id (DRPCExecutionException. "Request failed"))
            (.release sem)
            )))
      (^DRPCRequest fetchRequest [this ^String func]
        (let [^ConcurrentLinkedQueue queue (acquire-queue request-queues func)
              ret (.poll queue)]
          (if ret
            (do (log-debug "Fetched request for " func " at " (System/currentTimeMillis))
                ret)
            (DRPCRequest. "" ""))
          ))
      Shutdownable
      (shutdown [this]
        (.interrupt clear-thread))
      )))

(defn launch-server!
  ([]
    (let [conf (read-storm-config)
          worker-threads (int (conf DRPC-WORKER-THREADS))
          queue-size (int (conf DRPC-QUEUE-SIZE))
          service-handler (service-handler)
          ;; requests and returns need to be on separate thread pools, since calls to
          ;; "execute" don't unblock until other thrift methods are called. So if 
          ;; 64 threads are calling execute, the server won't accept the result
          ;; invocations that will unblock those threads
          handler-server (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-PORT)))
                                             (THsHaServer$Args.)
                                             (.workerThreads 64)
                                             (.executorService (ThreadPoolExecutor. worker-threads worker-threads 
                                                                 60 TimeUnit/SECONDS (ArrayBlockingQueue. queue-size)))
                                             (.protocolFactory (TBinaryProtocol$Factory.))
                                             (.processor (DistributedRPC$Processor. service-handler))
                                             ))
          invoke-server (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-INVOCATIONS-PORT)))
                                             (THsHaServer$Args.)
                                             (.workerThreads 64)
                                             (.protocolFactory (TBinaryProtocol$Factory.))
                                             (.processor (DistributedRPCInvocations$Processor. service-handler))
                                             ))]
      
      (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop handler-server) (.stop invoke-server))))
      (log-message "Starting Distributed RPC servers...")
      (future (.serve invoke-server))
      (.serve handler-server))))

(defn -main []
  (launch-server!))


================================================
FILE: storm-core/src/clj/backtype/storm/daemon/executor.clj
================================================
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements.  See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership.  The ASF licenses this file
;; to you under the Apache License, Version 2.0 (the
;; "License"); you may not use this file except in compliance
;; with the License.  You may obtain a copy of the License at
;;
;; http:;; www.apache.org/licenses/LICENSE-2.0
;;
;; Unless required by applicable law or agreed to in writing, software
;; distributed under the License is distributed on an "AS IS" BASIS,
;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.executor
  (:use [backtype.storm.daemon common])
  (:use [backtype.storm bootstrap])
  (:import [backtype.storm.hooks ITaskHook])
  (:import [backtype.storm.tuple Tuple])
  (:import [backtype.storm.spout ISpoutWaitStrategy])
  (:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo
            EmitInfo BoltFailInfo BoltAckInfo BoltExecuteInfo])
  (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
  (:import [backtype.storm Config])
  (:require [backtype.storm [tuple :as tuple]])
  (:require [backtype.storm.daemon [task :as task]])
  (:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics]))

(bootstrap)

(defn- mk-fields-grouper [^Fields out-fields ^Fields group-fields ^List target-tasks]
  (let [num-tasks (count target-tasks)
        task-getter (fn [i] (.get target-tasks i))]
    (fn [task-id ^List values]
      (-> (.select out-fields group-fields values)
          tuple/list-hash-code
          (mod num-tasks)
          task-getter))))

(defn- mk-shuffle-grouper [^List target-tasks]
  (let [choices (rotating-random-range target-tasks)]
    (fn [task-id tuple]
      (acquire-random-range-id choices))))

(defn- mk-custom-grouper [^CustomStreamGrouping grouping ^WorkerTopologyContext context ^String component-id ^String stream-id target-tasks]
  (.prepare grouping context (GlobalStreamId. component-id stream-id) target-tasks)
  (fn [task-id ^List values]
    (.chooseTasks grouping task-id values)
    ))

(defn- mk-grouper
  "Returns a function that returns a vector of which task indices to send tuple to, or just a single task index."
  [^WorkerTopologyContext context component-id stream-id ^Fields out-fields thrift-grouping ^List target-tasks]
  (let [num-tasks (count target-tasks)
        random (Random.)
        target-tasks (vec (sort target-tasks))]
    (condp = (thrift/grouping-type thrift-grouping)
      :fields
        (if (thrift/global-grouping? thrift-grouping)
          (fn [task-id tuple]
            ;; It's possible for target to have multiple tasks if it reads multiple sources
            (first target-tasks))
          (let [group-fields (Fields. (thrift/field-grouping thrift-grouping))]
            (mk-fields-grouper out-fields group-fields target-tasks)
            ))
      :all
        (fn [task-id tuple] target-tasks)
      :shuffle
        (mk-shuffle-grouper target-tasks)
      :local-or-shuffle
        (let [same-tasks (set/intersection
                           (set target-tasks)
                           (set (.getThisWorkerTasks context)))]
          (if-not (empty? same-tasks)
            (mk-shuffle-grouper (vec same-tasks))
            (mk-shuffle-grouper target-tasks)))
      :none
        (fn [task-id tuple]
          (let [i (mod (.nextInt random) num-tasks)]
            (.get target-tasks i)
            ))
      :custom-object
        (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))]
          (mk-custom-grouper grouping context component-id stream-id target-tasks))
      :direct
        :direct
      )))

(defn- outbound-groupings [^WorkerTopologyContext worker-context this-component-id stream-id out-fields component->grouping]
  (->> component->grouping
       (filter-key #(-> worker-context
                        (.getComponentTasks %)
                        count
                        pos?))
       (map (fn [[component tgrouping]]
               [component
                (mk-grouper worker-context
                            this-component-id
                            stream-id
                            out-fields
                            tgrouping
                            (.getComponentTasks worker-context component)
                            )]))
       (into {})
       (HashMap.)))

(defn outbound-components
  "Returns map of stream id to component id to grouper"
  [^WorkerTopologyContext worker-context component-id]
  (->> (.getTargets worker-context component-id)
        clojurify-structure
        (map (fn [[stream-id component->grouping]]
               [stream-id
                (outbound-groupings
                  worker-context
                  component-id
                  stream-id
                  (.getComponentOutputFields worker-context component-id stream-id)
                  component->grouping)]))
         (into {})
         (HashMap.)))

(defn executor-type [^WorkerTopologyContext context component-id]
  (let [topology (.getRawTopology context)
        spouts (.get_spouts topology)
        bolts (.get_bolts topology)]
    (cond (contains? spouts component-id) :spout
          (contains? bolts component-id) :bolt
          :else (throw-runtime "Could not find " component-id " in topology " topology))))

(defn executor-selector [executor-data & _] (:type executor-data))

(defmulti mk-threads executor-selector)
(defmulti mk-executor-stats executor-selector)
(defmulti close-component executor-selector)

(defn- normalized-component-conf [storm-conf general-context component-id]
  (let [to-remove (disj (set ALL-CONFIGS)
                        TOPOLOGY-DEBUG
                        TOPOLOGY-MAX-SPOUT-PENDING
                        TOPOLOGY-MAX-TASK-PARALLELISM
                        TOPOLOGY-TRANSACTIONAL-ID
                        TOPOLOGY-TICK-TUPLE-FREQ-SECS
                        TOPOLOGY-SLEEP-SPOUT-WAIT-STRATEGY-TIME-MS
                        TOPOLOGY-SPOUT-WAIT-STRATEGY
                        )
        spec-conf (-> general-context
                      (.getComponentCommon component-id)
                      .get_json_conf
                      from-json)]
    (merge storm-conf (apply dissoc spec-conf to-remove))
    ))

(defprotocol RunningExecutor
  (render-stats [this])
  (get-executor-id [this]))

(defn throttled-report-error-fn [executor]
  (let [storm-conf (:storm-conf executor)
        error-interval-secs (storm-conf TOPOLOGY-ERROR-THROTTLE-INTERVAL-SECS)
        max-per-interval (storm-conf TOPOLOGY-MAX-ERROR-REPORT-PER-INTERVAL)
        interval-start-time (atom (current-time-secs))
        interval-errors (atom 0)
        ]
    (fn [error]
      (log-error error)
      (when (> (time-delta @interval-start-time)
               error-interval-secs)
        (reset! interval-errors 0)
        (reset! interval-start-time (current-time-secs)))
      (swap! interval-errors inc)

      (when (<= @interval-errors max-per-interval)
        (cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor) error)
        ))))

;; in its own function so that it can be mocked out by tracked topologies
(defn mk-executor-transfer-fn [batch-transfer->worker]
  (fn this
    ([task tuple block? ^List overflow-buffer]
      (if (and overflow-buffer (not (.isEmpty overflow-buffer)))
        (.add overflow-buffer [task tuple])
        (try-cause
          (disruptor/publish batch-transfer->worker [task tuple] block?)
        (catch InsufficientCapacityException e
          (if overflow-buffer
            (.add overflow-buffer [task tuple])
            (throw e))
          ))))
    ([task tuple overflow-buffer]
      (this task tuple (nil? overflow-buffer) overflow-buffer))
    ([task tuple]
      (this task tuple nil)
      )))

(defn mk-executor-data [worker executor-id]
  (let [worker-context (worker-context worker)
        task-ids (executor-id->tasks executor-id)
        component-id (.getComponentId worker-context (first task-ids))
        storm-conf (normalized-component-conf (:storm-conf worker) worker-context component-id)
        executor-type (executor-type worker-context component-id)
        batch-transfer->worker (disruptor/disruptor-queue
                                  (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
                                  :claim-strategy :single-threaded
                                  :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
        ]
    (recursive-map
     :worker worker
     :worker-context worker-context
     :executor-id executor-id
     :task-ids task-ids
     :component-id component-id
     :open-or-prepare-was-called? (atom false)
     :storm-conf storm-conf
     :receive-queue ((:executor-receive-queue-map worker) executor-id)
     :storm-id (:storm-id worker)
     :conf (:conf worker)
     :shared-executor-data (HashMap.)
     :storm-active-atom (:storm-active-atom worker)
     :batch-transfer-queue batch-transfer->worker
     :transfer-fn (mk-executor-transfer-fn batch-transfer->worker)
     :suicide-fn (:suicide-fn worker)
     :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker))
     :type executor-type
     ;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)
     :stats (mk-executor-stats <> (sampling-rate storm-conf))
     :interval->task->metric-registry (HashMap.)
     :task->component (:task->component worker)
     :stream->component->grouper (outbound-components worker-context component-id)
     :report-error (throttled-report-error-fn <>)
     :report-error-and-die (fn [error]
                             ((:report-error <>) error)
                             ((:suicide-fn <>)))
     :deserializer (KryoTupleDeserializer. storm-conf worker-context)
     :sampler (mk-stats-sampler storm-conf)
     ;; TODO: add in the executor-specific stuff in a :specific... or make a spout-data, bolt-data function?
     )))

(defn start-batch-transfer->worker-handler! [worker executor-data]
  (let [worker-transfer-fn (:transfer-fn worker)
        cached-emit (MutableObject. (ArrayList.))
        storm-conf (:storm-conf executor-data)
        serializer (KryoTupleSerializer. storm-conf (:worker-context executor-data))
        ]
    (disruptor/consume-loop*
   
Download .txt
gitextract_bzfpz0c3/

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

FILE: storm-core/src/dev/resources/tester_bolt.py
  class TesterBolt (line 24) | class TesterBolt(storm.Bolt):
    method initialize (line 25) | def initialize(self, conf, context):
    method process (line 28) | def process(self, tup):

FILE: storm-core/src/dev/resources/tester_bolt.rb
  class TesterBolt (line 21) | class TesterBolt < Storm::Bolt
    method prepare (line 22) | def prepare(conf, context)
    method process (line 26) | def process(tuple)

FILE: storm-core/src/dev/resources/tester_spout.py
  class TesterSpout (line 28) | class TesterSpout(Spout):
    method initialize (line 29) | def initialize(self, conf, context):
    method nextTuple (line 33) | def nextTuple(self):
    method ack (line 40) | def ack(self, id):
    method fail (line 43) | def fail(self, id):

FILE: storm-core/src/dev/resources/tester_spout.rb
  function random_word (line 23) | def random_word
  class TesterSpout (line 27) | class TesterSpout < Storm::Spout
    method open (line 30) | def open(conf, context)
    method nextTuple (line 36) | def nextTuple
    method ack (line 44) | def ack(id)
    method fail (line 48) | def fail(id)

FILE: storm-core/src/jvm/backtype/storm/Config.java
  class Config (line 45) | public class Config extends HashMap<String, Object> {
    method setDebug (line 823) | public static void setDebug(Map conf, boolean isOn) {
    method setDebug (line 827) | public void setDebug(boolean isOn) {
    method setOptimize (line 831) | @Deprecated
    method setNumWorkers (line 836) | public static void setNumWorkers(Map conf, int workers) {
    method setNumWorkers (line 840) | public void setNumWorkers(int workers) {
    method setNumAckers (line 844) | public static void setNumAckers(Map conf, int numExecutors) {
    method setNumAckers (line 848) | public void setNumAckers(int numExecutors) {
    method setMessageTimeoutSecs (line 852) | public static void setMessageTimeoutSecs(Map conf, int secs) {
    method setMessageTimeoutSecs (line 856) | public void setMessageTimeoutSecs(int secs) {
    method registerSerialization (line 860) | public static void registerSerialization(Map conf, Class klass) {
    method registerSerialization (line 864) | public void registerSerialization(Class klass) {
    method registerSerialization (line 868) | public static void registerSerialization(Map conf, Class klass, Class<...
    method registerSerialization (line 874) | public void registerSerialization(Class klass, Class<? extends Seriali...
    method registerMetricsConsumer (line 878) | public void registerMetricsConsumer(Class klass, Object argument, long...
    method registerMetricsConsumer (line 890) | public void registerMetricsConsumer(Class klass, long parallelismHint) {
    method registerMetricsConsumer (line 894) | public void registerMetricsConsumer(Class klass) {
    method registerDecorator (line 898) | public static void registerDecorator(Map conf, Class<? extends IKryoDe...
    method registerDecorator (line 902) | public void registerDecorator(Class<? extends IKryoDecorator> klass) {
    method setKryoFactory (line 906) | public static void setKryoFactory(Map conf, Class<? extends IKryoFacto...
    method setKryoFactory (line 910) | public void setKryoFactory(Class<? extends IKryoFactory> klass) {
    method setSkipMissingKryoRegistrations (line 914) | public static void setSkipMissingKryoRegistrations(Map conf, boolean s...
    method setSkipMissingKryoRegistrations (line 918) | public void setSkipMissingKryoRegistrations(boolean skip) {
    method setMaxTaskParallelism (line 922) | public static void setMaxTaskParallelism(Map conf, int max) {
    method setMaxTaskParallelism (line 926) | public void setMaxTaskParallelism(int max) {
    method setMaxSpoutPending (line 930) | public static void setMaxSpoutPending(Map conf, int max) {
    method setMaxSpoutPending (line 934) | public void setMaxSpoutPending(int max) {
    method setStatsSampleRate (line 938) | public static void setStatsSampleRate(Map conf, double rate) {
    method setStatsSampleRate (line 942) | public void setStatsSampleRate(double rate) {
    method setFallBackOnJavaSerialization (line 946) | public static void setFallBackOnJavaSerialization(Map conf, boolean fa...
    method setFallBackOnJavaSerialization (line 950) | public void setFallBackOnJavaSerialization(boolean fallback) {
    method getRegisteredSerializations (line 954) | private static List getRegisteredSerializations(Map conf) {
    method getRegisteredDecorators (line 965) | private static List getRegisteredDecorators(Map conf) {

FILE: storm-core/src/jvm/backtype/storm/ConfigValidation.java
  class ConfigValidation (line 24) | public class ConfigValidation {
    type FieldValidator (line 29) | public static interface FieldValidator {
      method validateField (line 36) | public void validateField(String name, Object field) throws IllegalA...
    method FieldListValidatorFactory (line 44) | static FieldValidator FieldListValidatorFactory(final Class cls) {
    method validateField (line 88) | @Override

FILE: storm-core/src/jvm/backtype/storm/Constants.java
  class Constants (line 24) | public class Constants {

FILE: storm-core/src/jvm/backtype/storm/ILocalCluster.java
  type ILocalCluster (line 33) | public interface ILocalCluster {
    method submitTopology (line 34) | void submitTopology(String topologyName, Map conf, StormTopology topol...
    method submitTopologyWithOpts (line 35) | void submitTopologyWithOpts(String topologyName, Map conf, StormTopolo...
    method killTopology (line 36) | void killTopology(String topologyName) throws NotAliveException;
    method killTopologyWithOpts (line 37) | void killTopologyWithOpts(String name, KillOptions options) throws Not...
    method activate (line 38) | void activate(String topologyName) throws NotAliveException;
    method deactivate (line 39) | void deactivate(String topologyName) throws NotAliveException;
    method rebalance (line 40) | void rebalance(String name, RebalanceOptions options) throws NotAliveE...
    method shutdown (line 41) | void shutdown();
    method getTopologyConf (line 42) | String getTopologyConf(String id);
    method getTopology (line 43) | StormTopology getTopology(String id);
    method getClusterInfo (line 44) | ClusterSummary getClusterInfo();
    method getTopologyInfo (line 45) | TopologyInfo getTopologyInfo(String id);
    method getState (line 46) | Map getState();

FILE: storm-core/src/jvm/backtype/storm/ILocalDRPC.java
  type ILocalDRPC (line 25) | public interface ILocalDRPC extends DistributedRPC.Iface, DistributedRPC...
    method getServiceId (line 26) | public String getServiceId();

FILE: storm-core/src/jvm/backtype/storm/StormSubmitter.java
  class StormSubmitter (line 37) | public class StormSubmitter {
    method setLocalNimbus (line 42) | public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) {
    method submitTopology (line 57) | public static void submitTopology(String name, Map stormConf, StormTop...
    method submitTopology (line 73) | public static void submitTopology(String name, Map stormConf, StormTop...
    method topologyNameExists (line 116) | private static boolean topologyNameExists(Map conf, String name) {
    method submitJar (line 136) | private static void submitJar(Map conf) {
    method submitJar (line 146) | public static String submitJar(Map conf, String localJar) {

FILE: storm-core/src/jvm/backtype/storm/clojure/ClojureBolt.java
  class ClojureBolt (line 40) | public class ClojureBolt implements IRichBolt, FinishedCallback {
    method ClojureBolt (line 48) | public ClojureBolt(List fnSpec, List confSpec, List<Object> params, Ma...
    method prepare (line 55) | @Override
    method execute (line 81) | @Override
    method cleanup (line 86) | @Override
    method declareOutputFields (line 95) | @Override
    method finishedId (line 103) | @Override
    method getComponentConfiguration (line 110) | @Override

FILE: storm-core/src/jvm/backtype/storm/clojure/ClojureSpout.java
  class ClojureSpout (line 37) | public class ClojureSpout implements IRichSpout {
    method ClojureSpout (line 45) | public ClojureSpout(List fnSpec, List confSpec, List<Object> params, M...
    method open (line 53) | @Override
    method close (line 79) | @Override
    method nextTuple (line 88) | @Override
    method ack (line 98) | @Override
    method fail (line 108) | @Override
    method declareOutputFields (line 118) | @Override
    method getComponentConfiguration (line 126) | @Override
    method activate (line 136) | @Override
    method deactivate (line 145) | @Override

FILE: storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java
  class RichShellBolt (line 27) | public class RichShellBolt extends ShellBolt implements IRichBolt {
    method RichShellBolt (line 30) | public RichShellBolt(String[] command, Map<String, StreamInfo> outputs) {
    method declareOutputFields (line 35) | @Override
    method getComponentConfiguration (line 47) | @Override

FILE: storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java
  class RichShellSpout (line 27) | public class RichShellSpout extends ShellSpout implements IRichSpout {
    method RichShellSpout (line 30) | public RichShellSpout(String[] command, Map<String, StreamInfo> output...
    method declareOutputFields (line 35) | @Override
    method getComponentConfiguration (line 47) | @Override

FILE: storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
  class BatchBoltExecutor (line 34) | public class BatchBoltExecutor implements IRichBolt, FinishedCallback, T...
    method BatchBoltExecutor (line 43) | public BatchBoltExecutor(IBatchBolt bolt) {
    method prepare (line 47) | @Override
    method execute (line 55) | @Override
    method cleanup (line 68) | @Override
    method finishedId (line 72) | @Override
    method timeoutId (line 79) | @Override
    method declareOutputFields (line 85) | @Override
    method getComponentConfiguration (line 90) | @Override
    method getBatchBolt (line 95) | private IBatchBolt getBatchBolt(Object id) {
    method newTransactionalBolt (line 105) | private IBatchBolt newTransactionalBolt() {

FILE: storm-core/src/jvm/backtype/storm/coordination/BatchOutputCollector.java
  class BatchOutputCollector (line 23) | public abstract class BatchOutputCollector {
    method emit (line 28) | public List<Integer> emit(List<Object> tuple) {
    method emit (line 32) | public abstract List<Integer> emit(String streamId, List<Object> tuple);
    method emitDirect (line 39) | public void emitDirect(int taskId, List<Object> tuple) {
    method emitDirect (line 43) | public abstract void emitDirect(int taskId, String streamId, List<Obje...
    method reportError (line 45) | public abstract void reportError(Throwable error);

FILE: storm-core/src/jvm/backtype/storm/coordination/BatchOutputCollectorImpl.java
  class BatchOutputCollectorImpl (line 24) | public class BatchOutputCollectorImpl extends BatchOutputCollector {
    method BatchOutputCollectorImpl (line 27) | public BatchOutputCollectorImpl(OutputCollector collector) {
    method emit (line 31) | @Override
    method emitDirect (line 36) | @Override
    method reportError (line 41) | @Override
    method ack (line 46) | public void ack(Tuple tup) {
    method fail (line 50) | public void fail(Tuple tup) {

FILE: storm-core/src/jvm/backtype/storm/coordination/BatchSubtopologyBuilder.java
  class BatchSubtopologyBuilder (line 40) | public class BatchSubtopologyBuilder {
    method BatchSubtopologyBuilder (line 45) | public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterB...
    method BatchSubtopologyBuilder (line 51) | public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterB...
    method getMasterDeclarer (line 55) | public BoltDeclarer getMasterDeclarer() {
    method setBolt (line 59) | public BoltDeclarer setBolt(String id, IBatchBolt bolt) {
    method setBolt (line 63) | public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number paralle...
    method setBolt (line 67) | public BoltDeclarer setBolt(String id, IBasicBolt bolt) {
    method setBolt (line 71) | public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number paralle...
    method setBolt (line 75) | private BoltDeclarer setBolt(String id, IRichBolt bolt, Number paralle...
    method extendTopology (line 83) | public void extendTopology(TopologyBuilder builder) {
    method componentBoltSubscriptions (line 122) | private Set<String> componentBoltSubscriptions(Component component) {
    class Component (line 130) | private static class Component {
      method Component (line 136) | public Component(IRichBolt bolt, Integer parallelism) {
    type InputDeclaration (line 142) | private static interface InputDeclaration {
      method declare (line 143) | void declare(InputDeclarer declarer);
      method getComponent (line 144) | String getComponent();
    class BoltDeclarerImpl (line 147) | private class BoltDeclarerImpl extends BaseConfigurationDeclarer<BoltD...
      method BoltDeclarerImpl (line 150) | public BoltDeclarerImpl(Component component) {
      method fieldsGrouping (line 154) | @Override
      method fieldsGrouping (line 170) | @Override
      method globalGrouping (line 186) | @Override
      method globalGrouping (line 202) | @Override
      method shuffleGrouping (line 218) | @Override
      method shuffleGrouping (line 234) | @Override
      method localOrShuffleGrouping (line 250) | @Override
      method localOrShuffleGrouping (line 266) | @Override
      method noneGrouping (line 282) | @Override
      method noneGrouping (line 298) | @Override
      method allGrouping (line 314) | @Override
      method allGrouping (line 330) | @Override
      method directGrouping (line 346) | @Override
      method directGrouping (line 362) | @Override
      method customGrouping (line 378) | @Override
      method customGrouping (line 394) | @Override
      method grouping (line 410) | @Override
      method addDeclaration (line 426) | private void addDeclaration(InputDeclaration declaration) {
      method addConfigurations (line 430) | @Override

FILE: storm-core/src/jvm/backtype/storm/coordination/CoordinatedBolt.java
  class CoordinatedBolt (line 51) | public class CoordinatedBolt implements IRichBolt {
    type FinishedCallback (line 54) | public static interface FinishedCallback {
      method finishedId (line 55) | void finishedId(Object id);
    type TimeoutCallback (line 58) | public static interface TimeoutCallback {
      method timeoutId (line 59) | void timeoutId(Object id);
    class SourceArgs (line 63) | public static class SourceArgs implements Serializable {
      method SourceArgs (line 66) | protected SourceArgs(boolean singleCount) {
      method single (line 70) | public static SourceArgs single() {
      method all (line 74) | public static SourceArgs all() {
      method toString (line 78) | @Override
    class CoordinatedOutputCollector (line 84) | public class CoordinatedOutputCollector implements IOutputCollector {
      method CoordinatedOutputCollector (line 87) | public CoordinatedOutputCollector(IOutputCollector delegate) {
      method emit (line 91) | public List<Integer> emit(String stream, Collection<Tuple> anchors, ...
      method emitDirect (line 97) | public void emitDirect(int task, String stream, Collection<Tuple> an...
      method ack (line 102) | public void ack(Tuple tuple) {
      method fail (line 117) | public void fail(Tuple tuple) {
      method reportError (line 128) | public void reportError(Throwable error) {
      method updateTaskCounts (line 133) | private void updateTaskCounts(Object id, List<Integer> tasks) {
    class TrackingInfo (line 155) | public static class TrackingInfo {
      method toString (line 165) | @Override
    class IdStreamSpec (line 176) | public static class IdStreamSpec implements Serializable {
      method getGlobalStreamId (line 179) | public GlobalStreamId getGlobalStreamId() {
      method makeDetectSpec (line 183) | public static IdStreamSpec makeDetectSpec(String component, String s...
      method IdStreamSpec (line 187) | protected IdStreamSpec(String component, String stream) {
    method CoordinatedBolt (line 192) | public CoordinatedBolt(IRichBolt delegate) {
    method CoordinatedBolt (line 196) | public CoordinatedBolt(IRichBolt delegate, String sourceComponent, Sou...
    method CoordinatedBolt (line 200) | public CoordinatedBolt(IRichBolt delegate, Map<String, SourceArgs> sou...
    method prepare (line 207) | public void prepare(Map config, TopologyContext context, OutputCollect...
    method checkFinishId (line 235) | private boolean checkFinishId(Tuple tup, TupleType type) {
    method execute (line 298) | public void execute(Tuple tuple) {
    method cleanup (line 330) | public void cleanup() {
    method declareOutputFields (line 335) | public void declareOutputFields(OutputFieldsDeclarer declarer) {
    method getComponentConfiguration (line 340) | @Override
    method singleSourceArgs (line 345) | private static Map<String, SourceArgs> singleSourceArgs(String sourceC...
    class TimeoutItems (line 351) | private class TimeoutItems implements TimeCacheMap.ExpiredCallback<Obj...
      method expire (line 352) | @Override
    method getTupleType (line 365) | private TupleType getTupleType(Tuple tuple) {
    type TupleType (line 377) | static enum TupleType {

FILE: storm-core/src/jvm/backtype/storm/coordination/IBatchBolt.java
  type IBatchBolt (line 26) | public interface IBatchBolt<T> extends Serializable, IComponent {
    method prepare (line 27) | void prepare(Map conf, TopologyContext context, BatchOutputCollector c...
    method execute (line 28) | void execute(Tuple tuple);
    method finishBatch (line 29) | void finishBatch();

FILE: storm-core/src/jvm/backtype/storm/daemon/Shutdownable.java
  type Shutdownable (line 20) | public interface Shutdownable {
    method shutdown (line 21) | public void shutdown();

FILE: storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
  class DRPCInvocationsClient (line 28) | public class DRPCInvocationsClient implements DistributedRPCInvocations....
    method DRPCInvocationsClient (line 34) | public DRPCInvocationsClient(String host, int port) {
    method connect (line 44) | private void connect() throws TException {
    method getHost (line 50) | public String getHost() {
    method getPort (line 54) | public int getPort() {
    method result (line 58) | public void result(String id, String result) throws TException {
    method fetchRequest (line 68) | public DRPCRequest fetchRequest(String func) throws TException {
    method failRequest (line 78) | public void failRequest(String id) throws TException {
    method close (line 88) | public void close() {

FILE: storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
  class DRPCSpout (line 41) | public class DRPCSpout extends BaseRichSpout {
    class DRPCMessageId (line 49) | private static class DRPCMessageId {
      method DRPCMessageId (line 53) | public DRPCMessageId(String id, int index) {
    method DRPCSpout (line 60) | public DRPCSpout(String function) {
    method DRPCSpout (line 64) | public DRPCSpout(String function, ILocalDRPC drpc) {
    method open (line 69) | @Override
    method close (line 93) | @Override
    method nextTuple (line 100) | @Override
    method ack (line 144) | @Override
    method fail (line 148) | @Override
    method declareOutputFields (line 165) | @Override

FILE: storm-core/src/jvm/backtype/storm/drpc/JoinResult.java
  class JoinResult (line 35) | public class JoinResult extends BaseRichBolt {
    method JoinResult (line 43) | public JoinResult(String returnComponent) {
    method prepare (line 47) | public void prepare(Map map, TopologyContext context, OutputCollector ...
    method execute (line 51) | public void execute(Tuple tuple) {
    method declareOutputFields (line 72) | public void declareOutputFields(OutputFieldsDeclarer declarer) {

FILE: storm-core/src/jvm/backtype/storm/drpc/KeyedFairBolt.java
  class KeyedFairBolt (line 33) | public class KeyedFairBolt implements IRichBolt, FinishedCallback {
    method KeyedFairBolt (line 39) | public KeyedFairBolt(IRichBolt delegate) {
    method KeyedFairBolt (line 43) | public KeyedFairBolt(IBasicBolt delegate) {
    method prepare (line 48) | public void prepare(Map stormConf, TopologyContext context, OutputColl...
    method execute (line 69) | public void execute(Tuple input) {
    method cleanup (line 74) | public void cleanup() {
    method declareOutputFields (line 79) | public void declareOutputFields(OutputFieldsDeclarer declarer) {
    method finishedId (line 83) | public void finishedId(Object id) {
    method getComponentConfiguration (line 89) | @Override

FILE: storm-core/src/jvm/backtype/storm/drpc/LinearDRPCInputDeclarer.java
  type LinearDRPCInputDeclarer (line 24) | public interface LinearDRPCInputDeclarer extends ComponentConfigurationD...
    method fieldsGrouping (line 25) | public LinearDRPCInputDeclarer fieldsGrouping(Fields fields);
    method fieldsGrouping (line 26) | public LinearDRPCInputDeclarer fieldsGrouping(String streamId, Fields ...
    method globalGrouping (line 28) | public LinearDRPCInputDeclarer globalGrouping();
    method globalGrouping (line 29) | public LinearDRPCInputDeclarer globalGrouping(String streamId);
    method shuffleGrouping (line 31) | public LinearDRPCInputDeclarer shuffleGrouping();
    method shuffleGrouping (line 32) | public LinearDRPCInputDeclarer shuffleGrouping(String streamId);
    method localOrShuffleGrouping (line 34) | public LinearDRPCInputDeclarer localOrShuffleGrouping();
    method localOrShuffleGrouping (line 35) | public LinearDRPCInputDeclarer localOrShuffleGrouping(String streamId);
    method noneGrouping (line 37) | public LinearDRPCInputDeclarer noneGrouping();
    method noneGrouping (line 38) | public LinearDRPCInputDeclarer noneGrouping(String streamId);
    method allGrouping (line 40) | public LinearDRPCInputDeclarer allGrouping();
    method allGrouping (line 41) | public LinearDRPCInputDeclarer allGrouping(String streamId);
    method directGrouping (line 43) | public LinearDRPCInputDeclarer directGrouping();
    method directGrouping (line 44) | public LinearDRPCInputDeclarer directGrouping(String streamId);
    method customGrouping (line 46) | public LinearDRPCInputDeclarer customGrouping(CustomStreamGrouping gro...
    method customGrouping (line 47) | public LinearDRPCInputDeclarer customGrouping(String streamId, CustomS...

FILE: storm-core/src/jvm/backtype/storm/drpc/LinearDRPCTopologyBuilder.java
  class LinearDRPCTopologyBuilder (line 47) | @Deprecated
    method LinearDRPCTopologyBuilder (line 53) | public LinearDRPCTopologyBuilder(String function) {
    method addBolt (line 57) | public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt, Number paralle...
    method addBolt (line 61) | public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt) {
    method addBolt (line 65) | @Deprecated
    method addBolt (line 73) | @Deprecated
    method addBolt (line 78) | public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt, Number paralle...
    method addBolt (line 82) | public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt) {
    method createLocalTopology (line 86) | public StormTopology createLocalTopology(ILocalDRPC drpc) {
    method createRemoteTopology (line 90) | public StormTopology createRemoteTopology() {
    method createTopology (line 95) | private StormTopology createTopology(DRPCSpout spout) {
    method boltId (line 169) | private static String boltId(int index) {
    class Component (line 173) | private static class Component {
      method Component (line 179) | public Component(IRichBolt bolt, int parallelism) {
    type InputDeclaration (line 186) | private static interface InputDeclaration {
      method declare (line 187) | public void declare(String prevComponent, InputDeclarer declarer);
    class InputDeclarerImpl (line 190) | private class InputDeclarerImpl extends BaseConfigurationDeclarer<Line...
      method InputDeclarerImpl (line 193) | public InputDeclarerImpl(Component component) {
      method fieldsGrouping (line 197) | @Override
      method fieldsGrouping (line 208) | @Override
      method globalGrouping (line 219) | @Override
      method globalGrouping (line 230) | @Override
      method shuffleGrouping (line 241) | @Override
      method shuffleGrouping (line 252) | @Override
      method localOrShuffleGrouping (line 263) | @Override
      method localOrShuffleGrouping (line 274) | @Override
      method noneGrouping (line 285) | @Override
      method noneGrouping (line 296) | @Override
      method allGrouping (line 307) | @Override
      method allGrouping (line 318) | @Override
      method directGrouping (line 329) | @Override
      method directGrouping (line 340) | @Override
      method customGrouping (line 351) | @Override
      method customGrouping (line 362) | @Override
      method addDeclaration (line 373) | private void addDeclaration(InputDeclaration declaration) {
      method addConfigurations (line 377) | @Override

FILE: storm-core/src/jvm/backtype/storm/drpc/PrepareRequest.java
  class PrepareRequest (line 32) | public class PrepareRequest extends BaseBasicBolt {
    method prepare (line 39) | @Override
    method execute (line 44) | @Override
    method declareOutputFields (line 54) | public void declareOutputFields(OutputFieldsDeclarer declarer) {

FILE: storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java
  class ReturnResults (line 39) | public class ReturnResults extends BaseRichBolt {
    method prepare (line 46) | @Override
    method execute (line 52) | @Override
    method cleanup (line 86) | @Override
    method declareOutputFields (line 93) | public void declareOutputFields(OutputFieldsDeclarer declarer) {

FILE: storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
  class AlreadyAliveException (line 41) | public class AlreadyAliveException extends Exception implements org.apac...
    type _Fields (line 49) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 63) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 76) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 85) | public static _Fields findByName(String name) {
      method _Fields (line 92) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 97) | public short getThriftFieldId() {
      method getFieldName (line 101) | public String getFieldName() {
    method AlreadyAliveException (line 117) | public AlreadyAliveException() {
    method AlreadyAliveException (line 120) | public AlreadyAliveException(
    method AlreadyAliveException (line 130) | public AlreadyAliveException(AlreadyAliveException other) {
    method deepCopy (line 136) | public AlreadyAliveException deepCopy() {
    method clear (line 140) | @Override
    method get_msg (line 145) | public String get_msg() {
    method set_msg (line 149) | public void set_msg(String msg) {
    method unset_msg (line 153) | public void unset_msg() {
    method is_set_msg (line 158) | public boolean is_set_msg() {
    method set_msg_isSet (line 162) | public void set_msg_isSet(boolean value) {
    method setFieldValue (line 168) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 181) | public Object getFieldValue(_Fields field) {
    method isSet (line 191) | public boolean isSet(_Fields field) {
    method equals (line 203) | @Override
    method equals (line 212) | public boolean equals(AlreadyAliveException that) {
    method hashCode (line 228) | @Override
    method compareTo (line 240) | public int compareTo(AlreadyAliveException other) {
    method fieldForId (line 261) | public _Fields fieldForId(int fieldId) {
    method read (line 265) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 291) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 304) | @Override
    method validate (line 320) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 328) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 336) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/Bolt.java
  class Bolt (line 41) | public class Bolt implements org.apache.thrift7.TBase<Bolt, Bolt._Fields...
    type _Fields (line 51) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 66) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 81) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 90) | public static _Fields findByName(String name) {
      method _Fields (line 97) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 102) | public short getThriftFieldId() {
      method getFieldName (line 106) | public String getFieldName() {
    method Bolt (line 124) | public Bolt() {
    method Bolt (line 127) | public Bolt(
    method Bolt (line 139) | public Bolt(Bolt other) {
    method deepCopy (line 148) | public Bolt deepCopy() {
    method clear (line 152) | @Override
    method get_bolt_object (line 158) | public ComponentObject get_bolt_object() {
    method set_bolt_object (line 162) | public void set_bolt_object(ComponentObject bolt_object) {
    method unset_bolt_object (line 166) | public void unset_bolt_object() {
    method is_set_bolt_object (line 171) | public boolean is_set_bolt_object() {
    method set_bolt_object_isSet (line 175) | public void set_bolt_object_isSet(boolean value) {
    method get_common (line 181) | public ComponentCommon get_common() {
    method set_common (line 185) | public void set_common(ComponentCommon common) {
    method unset_common (line 189) | public void unset_common() {
    method is_set_common (line 194) | public boolean is_set_common() {
    method set_common_isSet (line 198) | public void set_common_isSet(boolean value) {
    method setFieldValue (line 204) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 225) | public Object getFieldValue(_Fields field) {
    method isSet (line 238) | public boolean isSet(_Fields field) {
    method equals (line 252) | @Override
    method equals (line 261) | public boolean equals(Bolt that) {
    method hashCode (line 286) | @Override
    method compareTo (line 303) | public int compareTo(Bolt other) {
    method fieldForId (line 334) | public _Fields fieldForId(int fieldId) {
    method read (line 338) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 373) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 391) | @Override
    method validate (line 415) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 427) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 435) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/BoltStats.java
  class BoltStats (line 41) | public class BoltStats implements org.apache.thrift7.TBase<BoltStats, Bo...
    type _Fields (line 57) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 75) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 96) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 105) | public static _Fields findByName(String name) {
      method _Fields (line 112) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 117) | public short getThriftFieldId() {
      method getFieldName (line 121) | public String getFieldName() {
    method BoltStats (line 165) | public BoltStats() {
    method BoltStats (line 168) | public BoltStats(
    method BoltStats (line 186) | public BoltStats(BoltStats other) {
    method deepCopy (line 319) | public BoltStats deepCopy() {
    method clear (line 323) | @Override
    method get_acked_size (line 332) | public int get_acked_size() {
    method put_to_acked (line 336) | public void put_to_acked(String key, Map<GlobalStreamId,Long> val) {
    method get_acked (line 343) | public Map<String,Map<GlobalStreamId,Long>> get_acked() {
    method set_acked (line 347) | public void set_acked(Map<String,Map<GlobalStreamId,Long>> acked) {
    method unset_acked (line 351) | public void unset_acked() {
    method is_set_acked (line 356) | public boolean is_set_acked() {
    method set_acked_isSet (line 360) | public void set_acked_isSet(boolean value) {
    method get_failed_size (line 366) | public int get_failed_size() {
    method put_to_failed (line 370) | public void put_to_failed(String key, Map<GlobalStreamId,Long> val) {
    method get_failed (line 377) | public Map<String,Map<GlobalStreamId,Long>> get_failed() {
    method set_failed (line 381) | public void set_failed(Map<String,Map<GlobalStreamId,Long>> failed) {
    method unset_failed (line 385) | public void unset_failed() {
    method is_set_failed (line 390) | public boolean is_set_failed() {
    method set_failed_isSet (line 394) | public void set_failed_isSet(boolean value) {
    method get_process_ms_avg_size (line 400) | public int get_process_ms_avg_size() {
    method put_to_process_ms_avg (line 404) | public void put_to_process_ms_avg(String key, Map<GlobalStreamId,Doubl...
    method get_process_ms_avg (line 411) | public Map<String,Map<GlobalStreamId,Double>> get_process_ms_avg() {
    method set_process_ms_avg (line 415) | public void set_process_ms_avg(Map<String,Map<GlobalStreamId,Double>> ...
    method unset_process_ms_avg (line 419) | public void unset_process_ms_avg() {
    method is_set_process_ms_avg (line 424) | public boolean is_set_process_ms_avg() {
    method set_process_ms_avg_isSet (line 428) | public void set_process_ms_avg_isSet(boolean value) {
    method get_executed_size (line 434) | public int get_executed_size() {
    method put_to_executed (line 438) | public void put_to_executed(String key, Map<GlobalStreamId,Long> val) {
    method get_executed (line 445) | public Map<String,Map<GlobalStreamId,Long>> get_executed() {
    method set_executed (line 449) | public void set_executed(Map<String,Map<GlobalStreamId,Long>> executed) {
    method unset_executed (line 453) | public void unset_executed() {
    method is_set_executed (line 458) | public boolean is_set_executed() {
    method set_executed_isSet (line 462) | public void set_executed_isSet(boolean value) {
    method get_execute_ms_avg_size (line 468) | public int get_execute_ms_avg_size() {
    method put_to_execute_ms_avg (line 472) | public void put_to_execute_ms_avg(String key, Map<GlobalStreamId,Doubl...
    method get_execute_ms_avg (line 479) | public Map<String,Map<GlobalStreamId,Double>> get_execute_ms_avg() {
    method set_execute_ms_avg (line 483) | public void set_execute_ms_avg(Map<String,Map<GlobalStreamId,Double>> ...
    method unset_execute_ms_avg (line 487) | public void unset_execute_ms_avg() {
    method is_set_execute_ms_avg (line 492) | public boolean is_set_execute_ms_avg() {
    method set_execute_ms_avg_isSet (line 496) | public void set_execute_ms_avg_isSet(boolean value) {
    method setFieldValue (line 502) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 547) | public Object getFieldValue(_Fields field) {
    method isSet (line 569) | public boolean isSet(_Fields field) {
    method equals (line 589) | @Override
    method equals (line 598) | public boolean equals(BoltStats that) {
    method hashCode (line 650) | @Override
    method compareTo (line 682) | public int compareTo(BoltStats other) {
    method fieldForId (line 743) | public _Fields fieldForId(int fieldId) {
    method read (line 747) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 926) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 1039) | @Override
    method validate (line 1087) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 1111) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 1119) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
  class ClusterSummary (line 41) | public class ClusterSummary implements org.apache.thrift7.TBase<ClusterS...
    type _Fields (line 53) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 69) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 86) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 95) | public static _Fields findByName(String name) {
      method _Fields (line 102) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 107) | public short getThriftFieldId() {
      method getFieldName (line 111) | public String getFieldName() {
    method ClusterSummary (line 135) | public ClusterSummary() {
    method ClusterSummary (line 138) | public ClusterSummary(
    method ClusterSummary (line 153) | public ClusterSummary(ClusterSummary other) {
    method deepCopy (line 173) | public ClusterSummary deepCopy() {
    method clear (line 177) | @Override
    method get_supervisors_size (line 185) | public int get_supervisors_size() {
    method get_supervisors_iterator (line 189) | public java.util.Iterator<SupervisorSummary> get_supervisors_iterator() {
    method add_to_supervisors (line 193) | public void add_to_supervisors(SupervisorSummary elem) {
    method get_supervisors (line 200) | public List<SupervisorSummary> get_supervisors() {
    method set_supervisors (line 204) | public void set_supervisors(List<SupervisorSummary> supervisors) {
    method unset_supervisors (line 208) | public void unset_supervisors() {
    method is_set_supervisors (line 213) | public boolean is_set_supervisors() {
    method set_supervisors_isSet (line 217) | public void set_supervisors_isSet(boolean value) {
    method get_nimbus_uptime_secs (line 223) | public int get_nimbus_uptime_secs() {
    method set_nimbus_uptime_secs (line 227) | public void set_nimbus_uptime_secs(int nimbus_uptime_secs) {
    method unset_nimbus_uptime_secs (line 232) | public void unset_nimbus_uptime_secs() {
    method is_set_nimbus_uptime_secs (line 237) | public boolean is_set_nimbus_uptime_secs() {
    method set_nimbus_uptime_secs_isSet (line 241) | public void set_nimbus_uptime_secs_isSet(boolean value) {
    method get_topologies_size (line 245) | public int get_topologies_size() {
    method get_topologies_iterator (line 249) | public java.util.Iterator<TopologySummary> get_topologies_iterator() {
    method add_to_topologies (line 253) | public void add_to_topologies(TopologySummary elem) {
    method get_topologies (line 260) | public List<TopologySummary> get_topologies() {
    method set_topologies (line 264) | public void set_topologies(List<TopologySummary> topologies) {
    method unset_topologies (line 268) | public void unset_topologies() {
    method is_set_topologies (line 273) | public boolean is_set_topologies() {
    method set_topologies_isSet (line 277) | public void set_topologies_isSet(boolean value) {
    method setFieldValue (line 283) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 312) | public Object getFieldValue(_Fields field) {
    method isSet (line 328) | public boolean isSet(_Fields field) {
    method equals (line 344) | @Override
    method equals (line 353) | public boolean equals(ClusterSummary that) {
    method hashCode (line 387) | @Override
    method compareTo (line 409) | public int compareTo(ClusterSummary other) {
    method fieldForId (line 450) | public _Fields fieldForId(int fieldId) {
    method read (line 454) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 517) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 552) | @Override
    method validate (line 580) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 596) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 604) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
  class ComponentCommon (line 41) | public class ComponentCommon implements org.apache.thrift7.TBase<Compone...
    type _Fields (line 55) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 72) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 91) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 100) | public static _Fields findByName(String name) {
      method _Fields (line 107) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 112) | public short getThriftFieldId() {
      method getFieldName (line 116) | public String getFieldName() {
    method ComponentCommon (line 144) | public ComponentCommon() {
    method ComponentCommon (line 147) | public ComponentCommon(
    method ComponentCommon (line 159) | public ComponentCommon(ComponentCommon other) {
    method deepCopy (line 198) | public ComponentCommon deepCopy() {
    method clear (line 202) | @Override
    method get_inputs_size (line 211) | public int get_inputs_size() {
    method put_to_inputs (line 215) | public void put_to_inputs(GlobalStreamId key, Grouping val) {
    method get_inputs (line 222) | public Map<GlobalStreamId,Grouping> get_inputs() {
    method set_inputs (line 226) | public void set_inputs(Map<GlobalStreamId,Grouping> inputs) {
    method unset_inputs (line 230) | public void unset_inputs() {
    method is_set_inputs (line 235) | public boolean is_set_inputs() {
    method set_inputs_isSet (line 239) | public void set_inputs_isSet(boolean value) {
    method get_streams_size (line 245) | public int get_streams_size() {
    method put_to_streams (line 249) | public void put_to_streams(String key, StreamInfo val) {
    method get_streams (line 256) | public Map<String,StreamInfo> get_streams() {
    method set_streams (line 260) | public void set_streams(Map<String,StreamInfo> streams) {
    method unset_streams (line 264) | public void unset_streams() {
    method is_set_streams (line 269) | public boolean is_set_streams() {
    method set_streams_isSet (line 273) | public void set_streams_isSet(boolean value) {
    method get_parallelism_hint (line 279) | public int get_parallelism_hint() {
    method set_parallelism_hint (line 283) | public void set_parallelism_hint(int parallelism_hint) {
    method unset_parallelism_hint (line 288) | public void unset_parallelism_hint() {
    method is_set_parallelism_hint (line 293) | public boolean is_set_parallelism_hint() {
    method set_parallelism_hint_isSet (line 297) | public void set_parallelism_hint_isSet(boolean value) {
    method get_json_conf (line 301) | public String get_json_conf() {
    method set_json_conf (line 305) | public void set_json_conf(String json_conf) {
    method unset_json_conf (line 309) | public void unset_json_conf() {
    method is_set_json_conf (line 314) | public boolean is_set_json_conf() {
    method set_json_conf_isSet (line 318) | public void set_json_conf_isSet(boolean value) {
    method setFieldValue (line 324) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 361) | public Object getFieldValue(_Fields field) {
    method isSet (line 380) | public boolean isSet(_Fields field) {
    method equals (line 398) | @Override
    method equals (line 407) | public boolean equals(ComponentCommon that) {
    method hashCode (line 450) | @Override
    method compareTo (line 477) | public int compareTo(ComponentCommon other) {
    method fieldForId (line 528) | public _Fields fieldForId(int fieldId) {
    method read (line 532) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 607) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 653) | @Override
    method validate (line 693) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 705) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 713) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/ComponentObject.java
  class ComponentObject (line 41) | public class ComponentObject extends org.apache.thrift7.TUnion<Component...
    type _Fields (line 48) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 64) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 81) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 90) | public static _Fields findByName(String name) {
      method _Fields (line 97) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 102) | public short getThriftFieldId() {
      method getFieldName (line 106) | public String getFieldName() {
    method ComponentObject (line 124) | public ComponentObject() {
    method ComponentObject (line 128) | public ComponentObject(_Fields setField, Object value) {
    method ComponentObject (line 132) | public ComponentObject(ComponentObject other) {
    method deepCopy (line 135) | public ComponentObject deepCopy() {
    method serialized_java (line 139) | public static ComponentObject serialized_java(ByteBuffer value) {
    method serialized_java (line 145) | public static ComponentObject serialized_java(byte[] value) {
    method shell (line 151) | public static ComponentObject shell(ShellComponent value) {
    method java_object (line 157) | public static ComponentObject java_object(JavaObject value) {
    method checkType (line 164) | @Override
    method readValue (line 187) | @Override
    method writeValue (line 230) | @Override
    method getFieldDesc (line 250) | @Override
    method getStructDesc (line 264) | @Override
    method enumForId (line 269) | @Override
    method fieldForId (line 274) | public _Fields fieldForId(int fieldId) {
    method get_serialized_java (line 279) | public byte[] get_serialized_java() {
    method buffer_for_serialized_java (line 285) | public ByteBuffer buffer_for_serialized_java() {
    method set_serialized_java (line 293) | public void set_serialized_java(byte[] value) {
    method set_serialized_java (line 297) | public void set_serialized_java(ByteBuffer value) {
    method get_shell (line 303) | public ShellComponent get_shell() {
    method set_shell (line 311) | public void set_shell(ShellComponent value) {
    method get_java_object (line 317) | public JavaObject get_java_object() {
    method set_java_object (line 325) | public void set_java_object(JavaObject value) {
    method is_set_serialized_java (line 331) | public boolean is_set_serialized_java() {
    method is_set_shell (line 336) | public boolean is_set_shell() {
    method is_set_java_object (line 341) | public boolean is_set_java_object() {
    method equals (line 346) | public boolean equals(Object other) {
    method equals (line 354) | public boolean equals(ComponentObject other) {
    method compareTo (line 358) | @Override
    method hashCode (line 368) | @Override
    method writeObject (line 384) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 393) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
  class DRPCExecutionException (line 41) | public class DRPCExecutionException extends Exception implements org.apa...
    type _Fields (line 49) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 63) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 76) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 85) | public static _Fields findByName(String name) {
      method _Fields (line 92) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 97) | public short getThriftFieldId() {
      method getFieldName (line 101) | public String getFieldName() {
    method DRPCExecutionException (line 117) | public DRPCExecutionException() {
    method DRPCExecutionException (line 120) | public DRPCExecutionException(
    method DRPCExecutionException (line 130) | public DRPCExecutionException(DRPCExecutionException other) {
    method deepCopy (line 136) | public DRPCExecutionException deepCopy() {
    method clear (line 140) | @Override
    method get_msg (line 145) | public String get_msg() {
    method set_msg (line 149) | public void set_msg(String msg) {
    method unset_msg (line 153) | public void unset_msg() {
    method is_set_msg (line 158) | public boolean is_set_msg() {
    method set_msg_isSet (line 162) | public void set_msg_isSet(boolean value) {
    method setFieldValue (line 168) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 181) | public Object getFieldValue(_Fields field) {
    method isSet (line 191) | public boolean isSet(_Fields field) {
    method equals (line 203) | @Override
    method equals (line 212) | public boolean equals(DRPCExecutionException that) {
    method hashCode (line 228) | @Override
    method compareTo (line 240) | public int compareTo(DRPCExecutionException other) {
    method fieldForId (line 261) | public _Fields fieldForId(int fieldId) {
    method read (line 265) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 291) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 304) | @Override
    method validate (line 320) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 328) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 336) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
  class DRPCRequest (line 41) | public class DRPCRequest implements org.apache.thrift7.TBase<DRPCRequest...
    type _Fields (line 51) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 66) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 81) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 90) | public static _Fields findByName(String name) {
      method _Fields (line 97) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 102) | public short getThriftFieldId() {
      method getFieldName (line 106) | public String getFieldName() {
    method DRPCRequest (line 124) | public DRPCRequest() {
    method DRPCRequest (line 127) | public DRPCRequest(
    method DRPCRequest (line 139) | public DRPCRequest(DRPCRequest other) {
    method deepCopy (line 148) | public DRPCRequest deepCopy() {
    method clear (line 152) | @Override
    method get_func_args (line 158) | public String get_func_args() {
    method set_func_args (line 162) | public void set_func_args(String func_args) {
    method unset_func_args (line 166) | public void unset_func_args() {
    method is_set_func_args (line 171) | public boolean is_set_func_args() {
    method set_func_args_isSet (line 175) | public void set_func_args_isSet(boolean value) {
    method get_request_id (line 181) | public String get_request_id() {
    method set_request_id (line 185) | public void set_request_id(String request_id) {
    method unset_request_id (line 189) | public void unset_request_id() {
    method is_set_request_id (line 194) | public boolean is_set_request_id() {
    method set_request_id_isSet (line 198) | public void set_request_id_isSet(boolean value) {
    method setFieldValue (line 204) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 225) | public Object getFieldValue(_Fields field) {
    method isSet (line 238) | public boolean isSet(_Fields field) {
    method equals (line 252) | @Override
    method equals (line 261) | public boolean equals(DRPCRequest that) {
    method hashCode (line 286) | @Override
    method compareTo (line 303) | public int compareTo(DRPCRequest other) {
    method fieldForId (line 334) | public _Fields fieldForId(int fieldId) {
    method read (line 338) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 371) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 389) | @Override
    method validate (line 413) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 425) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 433) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
  class DistributedRPC (line 41) | public class DistributedRPC {
    type Iface (line 43) | public interface Iface {
      method execute (line 45) | public String execute(String functionName, String funcArgs) throws D...
    type AsyncIface (line 49) | public interface AsyncIface {
      method execute (line 51) | public void execute(String functionName, String funcArgs, org.apache...
    class Client (line 55) | public static class Client extends org.apache.thrift7.TServiceClient i...
      class Factory (line 56) | public static class Factory implements org.apache.thrift7.TServiceCl...
        method Factory (line 57) | public Factory() {}
        method getClient (line 58) | public Client getClient(org.apache.thrift7.protocol.TProtocol prot) {
        method getClient (line 61) | public Client getClient(org.apache.thrift7.protocol.TProtocol ipro...
      method Client (line 66) | public Client(org.apache.thrift7.protocol.TProtocol prot)
      method Client (line 71) | public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apach...
      method execute (line 75) | public String execute(String functionName, String funcArgs) throws D...
      method send_execute (line 81) | public void send_execute(String functionName, String funcArgs) throw...
      method recv_execute (line 89) | public String recv_execute() throws DRPCExecutionException, org.apac...
    class AsyncClient (line 103) | public static class AsyncClient extends org.apache.thrift7.async.TAsyn...
      class Factory (line 104) | public static class Factory implements org.apache.thrift7.async.TAsy...
        method Factory (line 107) | public Factory(org.apache.thrift7.async.TAsyncClientManager client...
        method getAsyncClient (line 111) | public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNo...
      method AsyncClient (line 116) | public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory prot...
      method execute (line 120) | public void execute(String functionName, String funcArgs, org.apache...
      class execute_call (line 127) | public static class execute_call extends org.apache.thrift7.async.TA...
        method execute_call (line 130) | public execute_call(String functionName, String funcArgs, org.apac...
        method write_args (line 136) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 145) | public String getResult() throws DRPCExecutionException, org.apach...
    class Processor (line 157) | public static class Processor<I extends Iface> extends org.apache.thri...
      method Processor (line 159) | public Processor(I iface) {
      method Processor (line 163) | protected Processor(I iface, Map<String,  org.apache.thrift7.Process...
      method getProcessMap (line 167) | private static <I extends Iface> Map<String,  org.apache.thrift7.Pro...
      class execute (line 172) | private static class execute<I extends Iface> extends org.apache.thr...
        method execute (line 173) | public execute() {
        method getEmptyArgsInstance (line 177) | protected execute_args getEmptyArgsInstance() {
        method getResult (line 181) | protected execute_result getResult(I iface, execute_args args) thr...
    class execute_args (line 194) | public static class execute_args implements org.apache.thrift7.TBase<e...
      type _Fields (line 204) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 219) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 234) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 243) | public static _Fields findByName(String name) {
        method _Fields (line 250) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 255) | public short getThriftFieldId() {
        method getFieldName (line 259) | public String getFieldName() {
      method execute_args (line 277) | public execute_args() {
      method execute_args (line 280) | public execute_args(
      method execute_args (line 292) | public execute_args(execute_args other) {
      method deepCopy (line 301) | public execute_args deepCopy() {
      method clear (line 305) | @Override
      method get_functionName (line 311) | public String get_functionName() {
      method set_functionName (line 315) | public void set_functionName(String functionName) {
      method unset_functionName (line 319) | public void unset_functionName() {
      method is_set_functionName (line 324) | public boolean is_set_functionName() {
      method set_functionName_isSet (line 328) | public void set_functionName_isSet(boolean value) {
      method get_funcArgs (line 334) | public String get_funcArgs() {
      method set_funcArgs (line 338) | public void set_funcArgs(String funcArgs) {
      method unset_funcArgs (line 342) | public void unset_funcArgs() {
      method is_set_funcArgs (line 347) | public boolean is_set_funcArgs() {
      method set_funcArgs_isSet (line 351) | public void set_funcArgs_isSet(boolean value) {
      method setFieldValue (line 357) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 378) | public Object getFieldValue(_Fields field) {
      method isSet (line 391) | public boolean isSet(_Fields field) {
      method equals (line 405) | @Override
      method equals (line 414) | public boolean equals(execute_args that) {
      method hashCode (line 439) | @Override
      method compareTo (line 456) | public int compareTo(execute_args other) {
      method fieldForId (line 487) | public _Fields fieldForId(int fieldId) {
      method read (line 491) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 524) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 542) | @Override
      method validate (line 566) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 570) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 578) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class execute_result (line 588) | public static class execute_result implements org.apache.thrift7.TBase...
      type _Fields (line 598) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 613) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 628) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 637) | public static _Fields findByName(String name) {
        method _Fields (line 644) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 649) | public short getThriftFieldId() {
        method getFieldName (line 653) | public String getFieldName() {
      method execute_result (line 671) | public execute_result() {
      method execute_result (line 674) | public execute_result(
      method execute_result (line 686) | public execute_result(execute_result other) {
      method deepCopy (line 695) | public execute_result deepCopy() {
      method clear (line 699) | @Override
      method get_success (line 705) | public String get_success() {
      method set_success (line 709) | public void set_success(String success) {
      method unset_success (line 713) | public void unset_success() {
      method is_set_success (line 718) | public boolean is_set_success() {
      method set_success_isSet (line 722) | public void set_success_isSet(boolean value) {
      method get_e (line 728) | public DRPCExecutionException get_e() {
      method set_e (line 732) | public void set_e(DRPCExecutionException e) {
      method unset_e (line 736) | public void unset_e() {
      method is_set_e (line 741) | public boolean is_set_e() {
      method set_e_isSet (line 745) | public void set_e_isSet(boolean value) {
      method setFieldValue (line 751) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 772) | public Object getFieldValue(_Fields field) {
      method isSet (line 785) | public boolean isSet(_Fields field) {
      method equals (line 799) | @Override
      method equals (line 808) | public boolean equals(execute_result that) {
      method hashCode (line 833) | @Override
      method compareTo (line 850) | public int compareTo(execute_result other) {
      method fieldForId (line 881) | public _Fields fieldForId(int fieldId) {
      method read (line 885) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 919) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 935) | @Override
      method validate (line 959) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 963) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 971) | private void readObject(java.io.ObjectInputStream in) throws java.io...

FILE: storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
  class DistributedRPCInvocations (line 41) | public class DistributedRPCInvocations {
    type Iface (line 43) | public interface Iface {
      method result (line 45) | public void result(String id, String result) throws org.apache.thrif...
      method fetchRequest (line 47) | public DRPCRequest fetchRequest(String functionName) throws org.apac...
      method failRequest (line 49) | public void failRequest(String id) throws org.apache.thrift7.TExcept...
    type AsyncIface (line 53) | public interface AsyncIface {
      method result (line 55) | public void result(String id, String result, org.apache.thrift7.asyn...
      method fetchRequest (line 57) | public void fetchRequest(String functionName, org.apache.thrift7.asy...
      method failRequest (line 59) | public void failRequest(String id, org.apache.thrift7.async.AsyncMet...
    class Client (line 63) | public static class Client extends org.apache.thrift7.TServiceClient i...
      class Factory (line 64) | public static class Factory implements org.apache.thrift7.TServiceCl...
        method Factory (line 65) | public Factory() {}
        method getClient (line 66) | public Client getClient(org.apache.thrift7.protocol.TProtocol prot) {
        method getClient (line 69) | public Client getClient(org.apache.thrift7.protocol.TProtocol ipro...
      method Client (line 74) | public Client(org.apache.thrift7.protocol.TProtocol prot)
      method Client (line 79) | public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apach...
      method result (line 83) | public void result(String id, String result) throws org.apache.thrif...
      method send_result (line 89) | public void send_result(String id, String result) throws org.apache....
      method recv_result (line 97) | public void recv_result() throws org.apache.thrift7.TException
      method fetchRequest (line 104) | public DRPCRequest fetchRequest(String functionName) throws org.apac...
      method send_fetchRequest (line 110) | public void send_fetchRequest(String functionName) throws org.apache...
      method recv_fetchRequest (line 117) | public DRPCRequest recv_fetchRequest() throws org.apache.thrift7.TEx...
      method failRequest (line 127) | public void failRequest(String id) throws org.apache.thrift7.TException
      method send_failRequest (line 133) | public void send_failRequest(String id) throws org.apache.thrift7.TE...
      method recv_failRequest (line 140) | public void recv_failRequest() throws org.apache.thrift7.TException
    class AsyncClient (line 148) | public static class AsyncClient extends org.apache.thrift7.async.TAsyn...
      class Factory (line 149) | public static class Factory implements org.apache.thrift7.async.TAsy...
        method Factory (line 152) | public Factory(org.apache.thrift7.async.TAsyncClientManager client...
        method getAsyncClient (line 156) | public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNo...
      method AsyncClient (line 161) | public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory prot...
      method result (line 165) | public void result(String id, String result, org.apache.thrift7.asyn...
      class result_call (line 172) | public static class result_call extends org.apache.thrift7.async.TAs...
        method result_call (line 175) | public result_call(String id, String result, org.apache.thrift7.as...
        method write_args (line 181) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 190) | public void getResult() throws org.apache.thrift7.TException {
      method fetchRequest (line 200) | public void fetchRequest(String functionName, org.apache.thrift7.asy...
      class fetchRequest_call (line 207) | public static class fetchRequest_call extends org.apache.thrift7.asy...
        method fetchRequest_call (line 209) | public fetchRequest_call(String functionName, org.apache.thrift7.a...
        method write_args (line 214) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 222) | public DRPCRequest getResult() throws org.apache.thrift7.TException {
      method failRequest (line 232) | public void failRequest(String id, org.apache.thrift7.async.AsyncMet...
      class failRequest_call (line 239) | public static class failRequest_call extends org.apache.thrift7.asyn...
        method failRequest_call (line 241) | public failRequest_call(String id, org.apache.thrift7.async.AsyncM...
        method write_args (line 246) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 254) | public void getResult() throws org.apache.thrift7.TException {
    class Processor (line 266) | public static class Processor<I extends Iface> extends org.apache.thri...
      method Processor (line 268) | public Processor(I iface) {
      method Processor (line 272) | protected Processor(I iface, Map<String,  org.apache.thrift7.Process...
      method getProcessMap (line 276) | private static <I extends Iface> Map<String,  org.apache.thrift7.Pro...
      class result (line 283) | private static class result<I extends Iface> extends org.apache.thri...
        method result (line 284) | public result() {
        method getEmptyArgsInstance (line 288) | protected result_args getEmptyArgsInstance() {
        method getResult (line 292) | protected result_result getResult(I iface, result_args args) throw...
      class fetchRequest (line 299) | private static class fetchRequest<I extends Iface> extends org.apach...
        method fetchRequest (line 300) | public fetchRequest() {
        method getEmptyArgsInstance (line 304) | protected fetchRequest_args getEmptyArgsInstance() {
        method getResult (line 308) | protected fetchRequest_result getResult(I iface, fetchRequest_args...
      class failRequest (line 315) | private static class failRequest<I extends Iface> extends org.apache...
        method failRequest (line 316) | public failRequest() {
        method getEmptyArgsInstance (line 320) | protected failRequest_args getEmptyArgsInstance() {
        method getResult (line 324) | protected failRequest_result getResult(I iface, failRequest_args a...
    class result_args (line 333) | public static class result_args implements org.apache.thrift7.TBase<re...
      type _Fields (line 343) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 358) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 373) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 382) | public static _Fields findByName(String name) {
        method _Fields (line 389) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 394) | public short getThriftFieldId() {
        method getFieldName (line 398) | public String getFieldName() {
      method result_args (line 416) | public result_args() {
      method result_args (line 419) | public result_args(
      method result_args (line 431) | public result_args(result_args other) {
      method deepCopy (line 440) | public result_args deepCopy() {
      method clear (line 444) | @Override
      method get_id (line 450) | public String get_id() {
      method set_id (line 454) | public void set_id(String id) {
      method unset_id (line 458) | public void unset_id() {
      method is_set_id (line 463) | public boolean is_set_id() {
      method set_id_isSet (line 467) | public void set_id_isSet(boolean value) {
      method get_result (line 473) | public String get_result() {
      method set_result (line 477) | public void set_result(String result) {
      method unset_result (line 481) | public void unset_result() {
      method is_set_result (line 486) | public boolean is_set_result() {
      method set_result_isSet (line 490) | public void set_result_isSet(boolean value) {
      method setFieldValue (line 496) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 517) | public Object getFieldValue(_Fields field) {
      method isSet (line 530) | public boolean isSet(_Fields field) {
      method equals (line 544) | @Override
      method equals (line 553) | public boolean equals(result_args that) {
      method hashCode (line 578) | @Override
      method compareTo (line 595) | public int compareTo(result_args other) {
      method fieldForId (line 626) | public _Fields fieldForId(int fieldId) {
      method read (line 630) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 663) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 681) | @Override
      method validate (line 705) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 709) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 717) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class result_result (line 727) | public static class result_result implements org.apache.thrift7.TBase<...
      type _Fields (line 733) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 747) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 758) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 767) | public static _Fields findByName(String name) {
        method _Fields (line 774) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 779) | public short getThriftFieldId() {
        method getFieldName (line 783) | public String getFieldName() {
      method result_result (line 794) | public result_result() {
      method result_result (line 800) | public result_result(result_result other) {
      method deepCopy (line 803) | public result_result deepCopy() {
      method clear (line 807) | @Override
      method setFieldValue (line 811) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 816) | public Object getFieldValue(_Fields field) {
      method isSet (line 823) | public boolean isSet(_Fields field) {
      method equals (line 833) | @Override
      method equals (line 842) | public boolean equals(result_result that) {
      method hashCode (line 849) | @Override
      method compareTo (line 856) | public int compareTo(result_result other) {
      method fieldForId (line 867) | public _Fields fieldForId(int fieldId) {
      method read (line 871) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 890) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 897) | @Override
      method validate (line 906) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 910) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 918) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class fetchRequest_args (line 928) | public static class fetchRequest_args implements org.apache.thrift7.TB...
      type _Fields (line 936) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 950) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 963) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 972) | public static _Fields findByName(String name) {
        method _Fields (line 979) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 984) | public short getThriftFieldId() {
        method getFieldName (line 988) | public String getFieldName() {
      method fetchRequest_args (line 1004) | public fetchRequest_args() {
      method fetchRequest_args (line 1007) | public fetchRequest_args(
      method fetchRequest_args (line 1017) | public fetchRequest_args(fetchRequest_args other) {
      method deepCopy (line 1023) | public fetchRequest_args deepCopy() {
      method clear (line 1027) | @Override
      method get_functionName (line 1032) | public String get_functionName() {
      method set_functionName (line 1036) | public void set_functionName(String functionName) {
      method unset_functionName (line 1040) | public void unset_functionName() {
      method is_set_functionName (line 1045) | public boolean is_set_functionName() {
      method set_functionName_isSet (line 1049) | public void set_functionName_isSet(boolean value) {
      method setFieldValue (line 1055) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 1068) | public Object getFieldValue(_Fields field) {
      method isSet (line 1078) | public boolean isSet(_Fields field) {
      method equals (line 1090) | @Override
      method equals (line 1099) | public boolean equals(fetchRequest_args that) {
      method hashCode (line 1115) | @Override
      method compareTo (line 1127) | public int compareTo(fetchRequest_args other) {
      method fieldForId (line 1148) | public _Fields fieldForId(int fieldId) {
      method read (line 1152) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 1178) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 1191) | @Override
      method validate (line 1207) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 1211) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 1219) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class fetchRequest_result (line 1229) | public static class fetchRequest_result implements org.apache.thrift7....
      type _Fields (line 1237) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 1251) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 1264) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 1273) | public static _Fields findByName(String name) {
        method _Fields (line 1280) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 1285) | public short getThriftFieldId() {
        method getFieldName (line 1289) | public String getFieldName() {
      method fetchRequest_result (line 1305) | public fetchRequest_result() {
      method fetchRequest_result (line 1308) | public fetchRequest_result(
      method fetchRequest_result (line 1318) | public fetchRequest_result(fetchRequest_result other) {
      method deepCopy (line 1324) | public fetchRequest_result deepCopy() {
      method clear (line 1328) | @Override
      method get_success (line 1333) | public DRPCRequest get_success() {
      method set_success (line 1337) | public void set_success(DRPCRequest success) {
      method unset_success (line 1341) | public void unset_success() {
      method is_set_success (line 1346) | public boolean is_set_success() {
      method set_success_isSet (line 1350) | public void set_success_isSet(boolean value) {
      method setFieldValue (line 1356) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 1369) | public Object getFieldValue(_Fields field) {
      method isSet (line 1379) | public boolean isSet(_Fields field) {
      method equals (line 1391) | @Override
      method equals (line 1400) | public boolean equals(fetchRequest_result that) {
      method hashCode (line 1416) | @Override
      method compareTo (line 1428) | public int compareTo(fetchRequest_result other) {
      method fieldForId (line 1449) | public _Fields fieldForId(int fieldId) {
      method read (line 1453) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 1480) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 1492) | @Override
      method validate (line 1508) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 1512) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 1520) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class failRequest_args (line 1530) | public static class failRequest_args implements org.apache.thrift7.TBa...
      type _Fields (line 1538) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 1552) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 1565) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 1574) | public static _Fields findByName(String name) {
        method _Fields (line 1581) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 1586) | public short getThriftFieldId() {
        method getFieldName (line 1590) | public String getFieldName() {
      method failRequest_args (line 1606) | public failRequest_args() {
      method failRequest_args (line 1609) | public failRequest_args(
      method failRequest_args (line 1619) | public failRequest_args(failRequest_args other) {
      method deepCopy (line 1625) | public failRequest_args deepCopy() {
      method clear (line 1629) | @Override
      method get_id (line 1634) | public String get_id() {
      method set_id (line 1638) | public void set_id(String id) {
      method unset_id (line 1642) | public void unset_id() {
      method is_set_id (line 1647) | public boolean is_set_id() {
      method set_id_isSet (line 1651) | public void set_id_isSet(boolean value) {
      method setFieldValue (line 1657) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 1670) | public Object getFieldValue(_Fields field) {
      method isSet (line 1680) | public boolean isSet(_Fields field) {
      method equals (line 1692) | @Override
      method equals (line 1701) | public boolean equals(failRequest_args that) {
      method hashCode (line 1717) | @Override
      method compareTo (line 1729) | public int compareTo(failRequest_args other) {
      method fieldForId (line 1750) | public _Fields fieldForId(int fieldId) {
      method read (line 1754) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 1780) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 1793) | @Override
      method validate (line 1809) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 1813) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 1821) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class failRequest_result (line 1831) | public static class failRequest_result implements org.apache.thrift7.T...
      type _Fields (line 1837) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 1851) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 1862) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 1871) | public static _Fields findByName(String name) {
        method _Fields (line 1878) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 1883) | public short getThriftFieldId() {
        method getFieldName (line 1887) | public String getFieldName() {
      method failRequest_result (line 1898) | public failRequest_result() {
      method failRequest_result (line 1904) | public failRequest_result(failRequest_result other) {
      method deepCopy (line 1907) | public failRequest_result deepCopy() {
      method clear (line 1911) | @Override
      method setFieldValue (line 1915) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 1920) | public Object getFieldValue(_Fields field) {
      method isSet (line 1927) | public boolean isSet(_Fields field) {
      method equals (line 1937) | @Override
      method equals (line 1946) | public boolean equals(failRequest_result that) {
      method hashCode (line 1953) | @Override
      method compareTo (line 1960) | public int compareTo(failRequest_result other) {
      method fieldForId (line 1971) | public _Fields fieldForId(int fieldId) {
      method read (line 1975) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 1994) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 2001) | @Override
      method validate (line 2010) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 2014) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 2022) | private void readObject(java.io.ObjectInputStream in) throws java.io...

FILE: storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
  class ErrorInfo (line 41) | public class ErrorInfo implements org.apache.thrift7.TBase<ErrorInfo, Er...
    type _Fields (line 51) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 66) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 81) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 90) | public static _Fields findByName(String name) {
      method _Fields (line 97) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 102) | public short getThriftFieldId() {
      method getFieldName (line 106) | public String getFieldName() {
    method ErrorInfo (line 126) | public ErrorInfo() {
    method ErrorInfo (line 129) | public ErrorInfo(
    method ErrorInfo (line 142) | public ErrorInfo(ErrorInfo other) {
    method deepCopy (line 151) | public ErrorInfo deepCopy() {
    method clear (line 155) | @Override
    method get_error (line 162) | public String get_error() {
    method set_error (line 166) | public void set_error(String error) {
    method unset_error (line 170) | public void unset_error() {
    method is_set_error (line 175) | public boolean is_set_error() {
    method set_error_isSet (line 179) | public void set_error_isSet(boolean value) {
    method get_error_time_secs (line 185) | public int get_error_time_secs() {
    method set_error_time_secs (line 189) | public void set_error_time_secs(int error_time_secs) {
    method unset_error_time_secs (line 194) | public void unset_error_time_secs() {
    method is_set_error_time_secs (line 199) | public boolean is_set_error_time_secs() {
    method set_error_time_secs_isSet (line 203) | public void set_error_time_secs_isSet(boolean value) {
    method setFieldValue (line 207) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 228) | public Object getFieldValue(_Fields field) {
    method isSet (line 241) | public boolean isSet(_Fields field) {
    method equals (line 255) | @Override
    method equals (line 264) | public boolean equals(ErrorInfo that) {
    method hashCode (line 289) | @Override
    method compareTo (line 306) | public int compareTo(ErrorInfo other) {
    method fieldForId (line 337) | public _Fields fieldForId(int fieldId) {
    method read (line 341) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 375) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 391) | @Override
    method validate (line 411) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 423) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 431) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
  class ExecutorInfo (line 41) | public class ExecutorInfo implements org.apache.thrift7.TBase<ExecutorIn...
    type _Fields (line 51) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 66) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 81) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 90) | public static _Fields findByName(String name) {
      method _Fields (line 97) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 102) | public short getThriftFieldId() {
      method getFieldName (line 106) | public String getFieldName() {
    method ExecutorInfo (line 127) | public ExecutorInfo() {
    method ExecutorInfo (line 130) | public ExecutorInfo(
    method ExecutorInfo (line 144) | public ExecutorInfo(ExecutorInfo other) {
    method deepCopy (line 151) | public ExecutorInfo deepCopy() {
    method clear (line 155) | @Override
    method get_task_start (line 163) | public int get_task_start() {
    method set_task_start (line 167) | public void set_task_start(int task_start) {
    method unset_task_start (line 172) | public void unset_task_start() {
    method is_set_task_start (line 177) | public boolean is_set_task_start() {
    method set_task_start_isSet (line 181) | public void set_task_start_isSet(boolean value) {
    method get_task_end (line 185) | public int get_task_end() {
    method set_task_end (line 189) | public void set_task_end(int task_end) {
    method unset_task_end (line 194) | public void unset_task_end() {
    method is_set_task_end (line 199) | public boolean is_set_task_end() {
    method set_task_end_isSet (line 203) | public void set_task_end_isSet(boolean value) {
    method setFieldValue (line 207) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 228) | public Object getFieldValue(_Fields field) {
    method isSet (line 241) | public boolean isSet(_Fields field) {
    method equals (line 255) | @Override
    method equals (line 264) | public boolean equals(ExecutorInfo that) {
    method hashCode (line 289) | @Override
    method compareTo (line 306) | public int compareTo(ExecutorInfo other) {
    method fieldForId (line 337) | public _Fields fieldForId(int fieldId) {
    method read (line 341) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 376) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 390) | @Override
    method validate (line 406) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 418) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 426) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorSpecificStats.java
  class ExecutorSpecificStats (line 41) | public class ExecutorSpecificStats extends org.apache.thrift7.TUnion<Exe...
    type _Fields (line 47) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 62) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 77) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 86) | public static _Fields findByName(String name) {
      method _Fields (line 93) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 98) | public short getThriftFieldId() {
      method getFieldName (line 102) | public String getFieldName() {
    method ExecutorSpecificStats (line 118) | public ExecutorSpecificStats() {
    method ExecutorSpecificStats (line 122) | public ExecutorSpecificStats(_Fields setField, Object value) {
    method ExecutorSpecificStats (line 126) | public ExecutorSpecificStats(ExecutorSpecificStats other) {
    method deepCopy (line 129) | public ExecutorSpecificStats deepCopy() {
    method bolt (line 133) | public static ExecutorSpecificStats bolt(BoltStats value) {
    method spout (line 139) | public static ExecutorSpecificStats spout(SpoutStats value) {
    method checkType (line 146) | @Override
    method readValue (line 164) | @Override
    method writeValue (line 198) | @Override
    method getFieldDesc (line 214) | @Override
    method getStructDesc (line 226) | @Override
    method enumForId (line 231) | @Override
    method fieldForId (line 236) | public _Fields fieldForId(int fieldId) {
    method get_bolt (line 241) | public BoltStats get_bolt() {
    method set_bolt (line 249) | public void set_bolt(BoltStats value) {
    method get_spout (line 255) | public SpoutStats get_spout() {
    method set_spout (line 263) | public void set_spout(SpoutStats value) {
    method is_set_bolt (line 269) | public boolean is_set_bolt() {
    method is_set_spout (line 274) | public boolean is_set_spout() {
    method equals (line 279) | public boolean equals(Object other) {
    method equals (line 287) | public boolean equals(ExecutorSpecificStats other) {
    method compareTo (line 291) | @Override
    method hashCode (line 301) | @Override
    method writeObject (line 317) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 326) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
  class ExecutorStats (line 41) | public class ExecutorStats implements org.apache.thrift7.TBase<ExecutorS...
    type _Fields (line 53) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 69) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 86) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 95) | public static _Fields findByName(String name) {
      method _Fields (line 102) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 107) | public short getThriftFieldId() {
      method getFieldName (line 111) | public String getFieldName() {
    method ExecutorStats (line 139) | public ExecutorStats() {
    method ExecutorStats (line 142) | public ExecutorStats(
    method ExecutorStats (line 156) | public ExecutorStats(ExecutorStats other) {
    method deepCopy (line 214) | public ExecutorStats deepCopy() {
    method clear (line 218) | @Override
    method get_emitted_size (line 225) | public int get_emitted_size() {
    method put_to_emitted (line 229) | public void put_to_emitted(String key, Map<String,Long> val) {
    method get_emitted (line 236) | public Map<String,Map<String,Long>> get_emitted() {
    method set_emitted (line 240) | public void set_emitted(Map<String,Map<String,Long>> emitted) {
    method unset_emitted (line 244) | public void unset_emitted() {
    method is_set_emitted (line 249) | public boolean is_set_emitted() {
    method set_emitted_isSet (line 253) | public void set_emitted_isSet(boolean value) {
    method get_transferred_size (line 259) | public int get_transferred_size() {
    method put_to_transferred (line 263) | public void put_to_transferred(String key, Map<String,Long> val) {
    method get_transferred (line 270) | public Map<String,Map<String,Long>> get_transferred() {
    method set_transferred (line 274) | public void set_transferred(Map<String,Map<String,Long>> transferred) {
    method unset_transferred (line 278) | public void unset_transferred() {
    method is_set_transferred (line 283) | public boolean is_set_transferred() {
    method set_transferred_isSet (line 287) | public void set_transferred_isSet(boolean value) {
    method get_specific (line 293) | public ExecutorSpecificStats get_specific() {
    method set_specific (line 297) | public void set_specific(ExecutorSpecificStats specific) {
    method unset_specific (line 301) | public void unset_specific() {
    method is_set_specific (line 306) | public boolean is_set_specific() {
    method set_specific_isSet (line 310) | public void set_specific_isSet(boolean value) {
    method setFieldValue (line 316) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 345) | public Object getFieldValue(_Fields field) {
    method isSet (line 361) | public boolean isSet(_Fields field) {
    method equals (line 377) | @Override
    method equals (line 386) | public boolean equals(ExecutorStats that) {
    method hashCode (line 420) | @Override
    method compareTo (line 442) | public int compareTo(ExecutorStats other) {
    method fieldForId (line 483) | public _Fields fieldForId(int fieldId) {
    method read (line 487) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 576) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 631) | @Override
    method validate (line 663) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 679) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 687) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
  class ExecutorSummary (line 41) | public class ExecutorSummary implements org.apache.thrift7.TBase<Executo...
    type _Fields (line 59) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 78) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 101) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 110) | public static _Fields findByName(String name) {
      method _Fields (line 117) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 122) | public short getThriftFieldId() {
      method getFieldName (line 126) | public String getFieldName() {
    method ExecutorSummary (line 155) | public ExecutorSummary() {
    method ExecutorSummary (line 158) | public ExecutorSummary(
    method ExecutorSummary (line 178) | public ExecutorSummary(ExecutorSummary other) {
    method deepCopy (line 197) | public ExecutorSummary deepCopy() {
    method clear (line 201) | @Override
    method get_executor_info (line 213) | public ExecutorInfo get_executor_info() {
    method set_executor_info (line 217) | public void set_executor_info(ExecutorInfo executor_info) {
    method unset_executor_info (line 221) | public void unset_executor_info() {
    method is_set_executor_info (line 226) | public boolean is_set_executor_info() {
    method set_executor_info_isSet (line 230) | public void set_executor_info_isSet(boolean value) {
    method get_component_id (line 236) | public String get_component_id() {
    method set_component_id (line 240) | public void set_component_id(String component_id) {
    method unset_component_id (line 244) | public void unset_component_id() {
    method is_set_component_id (line 249) | public boolean is_set_component_id() {
    method set_component_id_isSet (line 253) | public void set_component_id_isSet(boolean value) {
    method get_host (line 259) | public String get_host() {
    method set_host (line 263) | public void set_host(String host) {
    method unset_host (line 267) | public void unset_host() {
    method is_set_host (line 272) | public boolean is_set_host() {
    method set_host_isSet (line 276) | public void set_host_isSet(boolean value) {
    method get_port (line 282) | public int get_port() {
    method set_port (line 286) | public void set_port(int port) {
    method unset_port (line 291) | public void unset_port() {
    method is_set_port (line 296) | public boolean is_set_port() {
    method set_port_isSet (line 300) | public void set_port_isSet(boolean value) {
    method get_uptime_secs (line 304) | public int get_uptime_secs() {
    method set_uptime_secs (line 308) | public void set_uptime_secs(int uptime_secs) {
    method unset_uptime_secs (line 313) | public void unset_uptime_secs() {
    method is_set_uptime_secs (line 318) | public boolean is_set_uptime_secs() {
    method set_uptime_secs_isSet (line 322) | public void set_uptime_secs_isSet(boolean value) {
    method get_stats (line 326) | public ExecutorStats get_stats() {
    method set_stats (line 330) | public void set_stats(ExecutorStats stats) {
    method unset_stats (line 334) | public void unset_stats() {
    method is_set_stats (line 339) | public boolean is_set_stats() {
    method set_stats_isSet (line 343) | public void set_stats_isSet(boolean value) {
    method setFieldValue (line 349) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 402) | public Object getFieldValue(_Fields field) {
    method isSet (line 427) | public boolean isSet(_Fields field) {
    method equals (line 449) | @Override
    method equals (line 458) | public boolean equals(ExecutorSummary that) {
    method hashCode (line 519) | @Override
    method compareTo (line 556) | public int compareTo(ExecutorSummary other) {
    method fieldForId (line 627) | public _Fields fieldForId(int fieldId) {
    method read (line 631) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 696) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 732) | @Override
    method validate (line 782) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 806) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 814) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
  class GlobalStreamId (line 41) | public class GlobalStreamId implements org.apache.thrift7.TBase<GlobalSt...
    type _Fields (line 51) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 66) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 81) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 90) | public static _Fields findByName(String name) {
      method _Fields (line 97) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 102) | public short getThriftFieldId() {
      method getFieldName (line 106) | public String getFieldName() {
    method GlobalStreamId (line 124) | public GlobalStreamId() {
    method GlobalStreamId (line 127) | public GlobalStreamId(
    method GlobalStreamId (line 139) | public GlobalStreamId(GlobalStreamId other) {
    method deepCopy (line 148) | public GlobalStreamId deepCopy() {
    method clear (line 152) | @Override
    method get_componentId (line 158) | public String get_componentId() {
    method set_componentId (line 162) | public void set_componentId(String componentId) {
    method unset_componentId (line 166) | public void unset_componentId() {
    method is_set_componentId (line 171) | public boolean is_set_componentId() {
    method set_componentId_isSet (line 175) | public void set_componentId_isSet(boolean value) {
    method get_streamId (line 181) | public String get_streamId() {
    method set_streamId (line 185) | public void set_streamId(String streamId) {
    method unset_streamId (line 189) | public void unset_streamId() {
    method is_set_streamId (line 194) | public boolean is_set_streamId() {
    method set_streamId_isSet (line 198) | public void set_streamId_isSet(boolean value) {
    method setFieldValue (line 204) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 225) | public Object getFieldValue(_Fields field) {
    method isSet (line 238) | public boolean isSet(_Fields field) {
    method equals (line 252) | @Override
    method equals (line 261) | public boolean equals(GlobalStreamId that) {
    method hashCode (line 286) | @Override
    method compareTo (line 303) | public int compareTo(GlobalStreamId other) {
    method fieldForId (line 334) | public _Fields fieldForId(int fieldId) {
    method read (line 338) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 371) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 389) | @Override
    method validate (line 413) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 425) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 433) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/Grouping.java
  class Grouping (line 41) | public class Grouping extends org.apache.thrift7.TUnion<Grouping, Groupi...
    type _Fields (line 53) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 74) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 101) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 110) | public static _Fields findByName(String name) {
      method _Fields (line 117) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 122) | public short getThriftFieldId() {
      method getFieldName (line 126) | public String getFieldName() {
    method Grouping (line 155) | public Grouping() {
    method Grouping (line 159) | public Grouping(_Fields setField, Object value) {
    method Grouping (line 163) | public Grouping(Grouping other) {
    method deepCopy (line 166) | public Grouping deepCopy() {
    method fields (line 170) | public static Grouping fields(List<String> value) {
    method shuffle (line 176) | public static Grouping shuffle(NullStruct value) {
    method all (line 182) | public static Grouping all(NullStruct value) {
    method none (line 188) | public static Grouping none(NullStruct value) {
    method direct (line 194) | public static Grouping direct(NullStruct value) {
    method custom_object (line 200) | public static Grouping custom_object(JavaObject value) {
    method custom_serialized (line 206) | public static Grouping custom_serialized(ByteBuffer value) {
    method custom_serialized (line 212) | public static Grouping custom_serialized(byte[] value) {
    method local_or_shuffle (line 218) | public static Grouping local_or_shuffle(NullStruct value) {
    method checkType (line 225) | @Override
    method readValue (line 273) | @Override
    method writeValue (line 375) | @Override
    method getFieldDesc (line 422) | @Override
    method getStructDesc (line 446) | @Override
    method enumForId (line 451) | @Override
    method fieldForId (line 456) | public _Fields fieldForId(int fieldId) {
    method get_fields (line 461) | public List<String> get_fields() {
    method set_fields (line 469) | public void set_fields(List<String> value) {
    method get_shuffle (line 475) | public NullStruct get_shuffle() {
    method set_shuffle (line 483) | public void set_shuffle(NullStruct value) {
    method get_all (line 489) | public NullStruct get_all() {
    method set_all (line 497) | public void set_all(NullStruct value) {
    method get_none (line 503) | public NullStruct get_none() {
    method set_none (line 511) | public void set_none(NullStruct value) {
    method get_direct (line 517) | public NullStruct get_direct() {
    method set_direct (line 525) | public void set_direct(NullStruct value) {
    method get_custom_object (line 531) | public JavaObject get_custom_object() {
    method set_custom_object (line 539) | public void set_custom_object(JavaObject value) {
    method get_custom_serialized (line 545) | public byte[] get_custom_serialized() {
    method buffer_for_custom_serialized (line 551) | public ByteBuffer buffer_for_custom_serialized() {
    method set_custom_serialized (line 559) | public void set_custom_serialized(byte[] value) {
    method set_custom_serialized (line 563) | public void set_custom_serialized(ByteBuffer value) {
    method get_local_or_shuffle (line 569) | public NullStruct get_local_or_shuffle() {
    method set_local_or_shuffle (line 577) | public void set_local_or_shuffle(NullStruct value) {
    method is_set_fields (line 583) | public boolean is_set_fields() {
    method is_set_shuffle (line 588) | public boolean is_set_shuffle() {
    method is_set_all (line 593) | public boolean is_set_all() {
    method is_set_none (line 598) | public boolean is_set_none() {
    method is_set_direct (line 603) | public boolean is_set_direct() {
    method is_set_custom_object (line 608) | public boolean is_set_custom_object() {
    method is_set_custom_serialized (line 613) | public boolean is_set_custom_serialized() {
    method is_set_local_or_shuffle (line 618) | public boolean is_set_local_or_shuffle() {
    method equals (line 623) | public boolean equals(Object other) {
    method equals (line 631) | public boolean equals(Grouping other) {
    method compareTo (line 635) | @Override
    method hashCode (line 645) | @Override
    method writeObject (line 661) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 670) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
  class InvalidTopologyException (line 41) | public class InvalidTopologyException extends Exception implements org.a...
    type _Fields (line 49) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 63) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 76) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 85) | public static _Fields findByName(String name) {
      method _Fields (line 92) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 97) | public short getThriftFieldId() {
      method getFieldName (line 101) | public String getFieldName() {
    method InvalidTopologyException (line 117) | public InvalidTopologyException() {
    method InvalidTopologyException (line 120) | public InvalidTopologyException(
    method InvalidTopologyException (line 130) | public InvalidTopologyException(InvalidTopologyException other) {
    method deepCopy (line 136) | public InvalidTopologyException deepCopy() {
    method clear (line 140) | @Override
    method get_msg (line 145) | public String get_msg() {
    method set_msg (line 149) | public void set_msg(String msg) {
    method unset_msg (line 153) | public void unset_msg() {
    method is_set_msg (line 158) | public boolean is_set_msg() {
    method set_msg_isSet (line 162) | public void set_msg_isSet(boolean value) {
    method setFieldValue (line 168) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 181) | public Object getFieldValue(_Fields field) {
    method isSet (line 191) | public boolean isSet(_Fields field) {
    method equals (line 203) | @Override
    method equals (line 212) | public boolean equals(InvalidTopologyException that) {
    method hashCode (line 228) | @Override
    method compareTo (line 240) | public int compareTo(InvalidTopologyException other) {
    method fieldForId (line 261) | public _Fields fieldForId(int fieldId) {
    method read (line 265) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 291) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 304) | @Override
    method validate (line 320) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 328) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 336) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/JavaObject.java
  class JavaObject (line 41) | public class JavaObject implements org.apache.thrift7.TBase<JavaObject, ...
    type _Fields (line 51) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 66) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 81) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 90) | public static _Fields findByName(String name) {
      method _Fields (line 97) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 102) | public short getThriftFieldId() {
      method getFieldName (line 106) | public String getFieldName() {
    method JavaObject (line 125) | public JavaObject() {
    method JavaObject (line 128) | public JavaObject(
    method JavaObject (line 140) | public JavaObject(JavaObject other) {
    method deepCopy (line 153) | public JavaObject deepCopy() {
    method clear (line 157) | @Override
    method get_full_class_name (line 163) | public String get_full_class_name() {
    method set_full_class_name (line 167) | public void set_full_class_name(String full_class_name) {
    method unset_full_class_name (line 171) | public void unset_full_class_name() {
    method is_set_full_class_name (line 176) | public boolean is_set_full_class_name() {
    method set_full_class_name_isSet (line 180) | public void set_full_class_name_isSet(boolean value) {
    method get_args_list_size (line 186) | public int get_args_list_size() {
    method get_args_list_iterator (line 190) | public java.util.Iterator<JavaObjectArg> get_args_list_iterator() {
    method add_to_args_list (line 194) | public void add_to_args_list(JavaObjectArg elem) {
    method get_args_list (line 201) | public List<JavaObjectArg> get_args_list() {
    method set_args_list (line 205) | public void set_args_list(List<JavaObjectArg> args_list) {
    method unset_args_list (line 209) | public void unset_args_list() {
    method is_set_args_list (line 214) | public boolean is_set_args_list() {
    method set_args_list_isSet (line 218) | public void set_args_list_isSet(boolean value) {
    method setFieldValue (line 224) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 245) | public Object getFieldValue(_Fields field) {
    method isSet (line 258) | public boolean isSet(_Fields field) {
    method equals (line 272) | @Override
    method equals (line 281) | public boolean equals(JavaObject that) {
    method hashCode (line 306) | @Override
    method compareTo (line 323) | public int compareTo(JavaObject other) {
    method fieldForId (line 354) | public _Fields fieldForId(int fieldId) {
    method read (line 358) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 402) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 427) | @Override
    method validate (line 451) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 463) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 471) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/JavaObjectArg.java
  class JavaObjectArg (line 41) | public class JavaObjectArg extends org.apache.thrift7.TUnion<JavaObjectA...
    type _Fields (line 51) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 70) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 93) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 102) | public static _Fields findByName(String name) {
      method _Fields (line 109) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 114) | public short getThriftFieldId() {
      method getFieldName (line 118) | public String getFieldName() {
    method JavaObjectArg (line 142) | public JavaObjectArg() {
    method JavaObjectArg (line 146) | public JavaObjectArg(_Fields setField, Object value) {
    method JavaObjectArg (line 150) | public JavaObjectArg(JavaObjectArg other) {
    method deepCopy (line 153) | public JavaObjectArg deepCopy() {
    method int_arg (line 157) | public static JavaObjectArg int_arg(int value) {
    method long_arg (line 163) | public static JavaObjectArg long_arg(long value) {
    method string_arg (line 169) | public static JavaObjectArg string_arg(String value) {
    method bool_arg (line 175) | public static JavaObjectArg bool_arg(boolean value) {
    method binary_arg (line 181) | public static JavaObjectArg binary_arg(ByteBuffer value) {
    method binary_arg (line 187) | public static JavaObjectArg binary_arg(byte[] value) {
    method double_arg (line 193) | public static JavaObjectArg double_arg(double value) {
    method checkType (line 200) | @Override
    method readValue (line 238) | @Override
    method writeValue (line 306) | @Override
    method getFieldDesc (line 338) | @Override
    method getStructDesc (line 358) | @Override
    method enumForId (line 363) | @Override
    method fieldForId (line 368) | public _Fields fieldForId(int fieldId) {
    method get_int_arg (line 373) | public int get_int_arg() {
    method set_int_arg (line 381) | public void set_int_arg(int value) {
    method get_long_arg (line 386) | public long get_long_arg() {
    method set_long_arg (line 394) | public void set_long_arg(long value) {
    method get_string_arg (line 399) | public String get_string_arg() {
    method set_string_arg (line 407) | public void set_string_arg(String value) {
    method get_bool_arg (line 413) | public boolean get_bool_arg() {
    method set_bool_arg (line 421) | public void set_bool_arg(boolean value) {
    method get_binary_arg (line 426) | public byte[] get_binary_arg() {
    method buffer_for_binary_arg (line 432) | public ByteBuffer buffer_for_binary_arg() {
    method set_binary_arg (line 440) | public void set_binary_arg(byte[] value) {
    method set_binary_arg (line 444) | public void set_binary_arg(ByteBuffer value) {
    method get_double_arg (line 450) | public double get_double_arg() {
    method set_double_arg (line 458) | public void set_double_arg(double value) {
    method is_set_int_arg (line 463) | public boolean is_set_int_arg() {
    method is_set_long_arg (line 468) | public boolean is_set_long_arg() {
    method is_set_string_arg (line 473) | public boolean is_set_string_arg() {
    method is_set_bool_arg (line 478) | public boolean is_set_bool_arg() {
    method is_set_binary_arg (line 483) | public boolean is_set_binary_arg() {
    method is_set_double_arg (line 488) | public boolean is_set_double_arg() {
    method equals (line 493) | public boolean equals(Object other) {
    method equals (line 501) | public boolean equals(JavaObjectArg other) {
    method compareTo (line 505) | @Override
    method hashCode (line 515) | @Override
    method writeObject (line 531) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 540) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/KillOptions.java
  class KillOptions (line 41) | public class KillOptions implements org.apache.thrift7.TBase<KillOptions...
    type _Fields (line 49) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
      method findByThriftId (line 63) | public static _Fields findByThriftId(int fieldId) {
      method findByThriftIdOrThrow (line 76) | public static _Fields findByThriftIdOrThrow(int fieldId) {
      method findByName (line 85) | public static _Fields findByName(String name) {
      method _Fields (line 92) | _Fields(short thriftId, String fieldName) {
      method getThriftFieldId (line 97) | public short getThriftFieldId() {
      method getFieldName (line 101) | public String getFieldName() {
    method KillOptions (line 119) | public KillOptions() {
    method KillOptions (line 125) | public KillOptions(KillOptions other) {
    method deepCopy (line 131) | public KillOptions deepCopy() {
    method clear (line 135) | @Override
    method get_wait_secs (line 141) | public int get_wait_secs() {
    method set_wait_secs (line 145) | public void set_wait_secs(int wait_secs) {
    method unset_wait_secs (line 150) | public void unset_wait_secs() {
    method is_set_wait_secs (line 155) | public boolean is_set_wait_secs() {
    method set_wait_secs_isSet (line 159) | public void set_wait_secs_isSet(boolean value) {
    method setFieldValue (line 163) | public void setFieldValue(_Fields field, Object value) {
    method getFieldValue (line 176) | public Object getFieldValue(_Fields field) {
    method isSet (line 186) | public boolean isSet(_Fields field) {
    method equals (line 198) | @Override
    method equals (line 207) | public boolean equals(KillOptions that) {
    method hashCode (line 223) | @Override
    method compareTo (line 235) | public int compareTo(KillOptions other) {
    method fieldForId (line 256) | public _Fields fieldForId(int fieldId) {
    method read (line 260) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws o...
    method write (line 287) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throws ...
    method toString (line 300) | @Override
    method validate (line 314) | public void validate() throws org.apache.thrift7.TException {
    method writeObject (line 318) | private void writeObject(java.io.ObjectOutputStream out) throws java.i...
    method readObject (line 326) | private void readObject(java.io.ObjectInputStream in) throws java.io.I...

FILE: storm-core/src/jvm/backtype/storm/generated/Nimbus.java
  class Nimbus (line 41) | public class Nimbus {
    type Iface (line 43) | public interface Iface {
      method submitTopology (line 45) | public void submitTopology(String name, String uploadedJarLocation, ...
      method submitTopologyWithOpts (line 47) | public void submitTopologyWithOpts(String name, String uploadedJarLo...
      method killTopology (line 49) | public void killTopology(String name) throws NotAliveException, org....
      method killTopologyWithOpts (line 51) | public void killTopologyWithOpts(String name, KillOptions options) t...
      method activate (line 53) | public void activate(String name) throws NotAliveException, org.apac...
      method deactivate (line 55) | public void deactivate(String name) throws NotAliveException, org.ap...
      method rebalance (line 57) | public void rebalance(String name, RebalanceOptions options) throws ...
      method beginFileUpload (line 59) | public String beginFileUpload() throws org.apache.thrift7.TException;
      method uploadChunk (line 61) | public void uploadChunk(String location, ByteBuffer chunk) throws or...
      method finishFileUpload (line 63) | public void finishFileUpload(String location) throws org.apache.thri...
      method beginFileDownload (line 65) | public String beginFileDownload(String file) throws org.apache.thrif...
      method downloadChunk (line 67) | public ByteBuffer downloadChunk(String id) throws org.apache.thrift7...
      method getNimbusConf (line 69) | public String getNimbusConf() throws org.apache.thrift7.TException;
      method getClusterInfo (line 71) | public ClusterSummary getClusterInfo() throws org.apache.thrift7.TEx...
      method getTopologyInfo (line 73) | public TopologyInfo getTopologyInfo(String id) throws NotAliveExcept...
      method getTopologyConf (line 75) | public String getTopologyConf(String id) throws NotAliveException, o...
      method getTopology (line 77) | public StormTopology getTopology(String id) throws NotAliveException...
      method getUserTopology (line 79) | public StormTopology getUserTopology(String id) throws NotAliveExcep...
    type AsyncIface (line 83) | public interface AsyncIface {
      method submitTopology (line 85) | public void submitTopology(String name, String uploadedJarLocation, ...
      method submitTopologyWithOpts (line 87) | public void submitTopologyWithOpts(String name, String uploadedJarLo...
      method killTopology (line 89) | public void killTopology(String name, org.apache.thrift7.async.Async...
      method killTopologyWithOpts (line 91) | public void killTopologyWithOpts(String name, KillOptions options, o...
      method activate (line 93) | public void activate(String name, org.apache.thrift7.async.AsyncMeth...
      method deactivate (line 95) | public void deactivate(String name, org.apache.thrift7.async.AsyncMe...
      method rebalance (line 97) | public void rebalance(String name, RebalanceOptions options, org.apa...
      method beginFileUpload (line 99) | public void beginFileUpload(org.apache.thrift7.async.AsyncMethodCall...
      method uploadChunk (line 101) | public void uploadChunk(String location, ByteBuffer chunk, org.apach...
      method finishFileUpload (line 103) | public void finishFileUpload(String location, org.apache.thrift7.asy...
      method beginFileDownload (line 105) | public void beginFileDownload(String file, org.apache.thrift7.async....
      method downloadChunk (line 107) | public void downloadChunk(String id, org.apache.thrift7.async.AsyncM...
      method getNimbusConf (line 109) | public void getNimbusConf(org.apache.thrift7.async.AsyncMethodCallba...
      method getClusterInfo (line 111) | public void getClusterInfo(org.apache.thrift7.async.AsyncMethodCallb...
      method getTopologyInfo (line 113) | public void getTopologyInfo(String id, org.apache.thrift7.async.Asyn...
      method getTopologyConf (line 115) | public void getTopologyConf(String id, org.apache.thrift7.async.Asyn...
      method getTopology (line 117) | public void getTopology(String id, org.apache.thrift7.async.AsyncMet...
      method getUserTopology (line 119) | public void getUserTopology(String id, org.apache.thrift7.async.Asyn...
    class Client (line 123) | public static class Client extends org.apache.thrift7.TServiceClient i...
      class Factory (line 124) | public static class Factory implements org.apache.thrift7.TServiceCl...
        method Factory (line 125) | public Factory() {}
        method getClient (line 126) | public Client getClient(org.apache.thrift7.protocol.TProtocol prot) {
        method getClient (line 129) | public Client getClient(org.apache.thrift7.protocol.TProtocol ipro...
      method Client (line 134) | public Client(org.apache.thrift7.protocol.TProtocol prot)
      method Client (line 139) | public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apach...
      method submitTopology (line 143) | public void submitTopology(String name, String uploadedJarLocation, ...
      method send_submitTopology (line 149) | public void send_submitTopology(String name, String uploadedJarLocat...
      method recv_submitTopology (line 159) | public void recv_submitTopology() throws AlreadyAliveException, Inva...
      method submitTopologyWithOpts (line 172) | public void submitTopologyWithOpts(String name, String uploadedJarLo...
      method send_submitTopologyWithOpts (line 178) | public void send_submitTopologyWithOpts(String name, String uploaded...
      method recv_submitTopologyWithOpts (line 189) | public void recv_submitTopologyWithOpts() throws AlreadyAliveExcepti...
      method killTopology (line 202) | public void killTopology(String name) throws NotAliveException, org....
      method send_killTopology (line 208) | public void send_killTopology(String name) throws org.apache.thrift7...
      method recv_killTopology (line 215) | public void recv_killTopology() throws NotAliveException, org.apache...
      method killTopologyWithOpts (line 225) | public void killTopologyWithOpts(String name, KillOptions options) t...
      method send_killTopologyWithOpts (line 231) | public void send_killTopologyWithOpts(String name, KillOptions optio...
      method recv_killTopologyWithOpts (line 239) | public void recv_killTopologyWithOpts() throws NotAliveException, or...
      method activate (line 249) | public void activate(String name) throws NotAliveException, org.apac...
      method send_activate (line 255) | public void send_activate(String name) throws org.apache.thrift7.TEx...
      method recv_activate (line 262) | public void recv_activate() throws NotAliveException, org.apache.thr...
      method deactivate (line 272) | public void deactivate(String name) throws NotAliveException, org.ap...
      method send_deactivate (line 278) | public void send_deactivate(String name) throws org.apache.thrift7.T...
      method recv_deactivate (line 285) | public void recv_deactivate() throws NotAliveException, org.apache.t...
      method rebalance (line 295) | public void rebalance(String name, RebalanceOptions options) throws ...
      method send_rebalance (line 301) | public void send_rebalance(String name, RebalanceOptions options) th...
      method recv_rebalance (line 309) | public void recv_rebalance() throws NotAliveException, InvalidTopolo...
      method beginFileUpload (line 322) | public String beginFileUpload() throws org.apache.thrift7.TException
      method send_beginFileUpload (line 328) | public void send_beginFileUpload() throws org.apache.thrift7.TException
      method recv_beginFileUpload (line 334) | public String recv_beginFileUpload() throws org.apache.thrift7.TExce...
      method uploadChunk (line 344) | public void uploadChunk(String location, ByteBuffer chunk) throws or...
      method send_uploadChunk (line 350) | public void send_uploadChunk(String location, ByteBuffer chunk) thro...
      method recv_uploadChunk (line 358) | public void recv_uploadChunk() throws org.apache.thrift7.TException
      method finishFileUpload (line 365) | public void finishFileUpload(String location) throws org.apache.thri...
      method send_finishFileUpload (line 371) | public void send_finishFileUpload(String location) throws org.apache...
      method recv_finishFileUpload (line 378) | public void recv_finishFileUpload() throws org.apache.thrift7.TExcep...
      method beginFileDownload (line 385) | public String beginFileDownload(String file) throws org.apache.thrif...
      method send_beginFileDownload (line 391) | public void send_beginFileDownload(String file) throws org.apache.th...
      method recv_beginFileDownload (line 398) | public String recv_beginFileDownload() throws org.apache.thrift7.TEx...
      method downloadChunk (line 408) | public ByteBuffer downloadChunk(String id) throws org.apache.thrift7...
      method send_downloadChunk (line 414) | public void send_downloadChunk(String id) throws org.apache.thrift7....
      method recv_downloadChunk (line 421) | public ByteBuffer recv_downloadChunk() throws org.apache.thrift7.TEx...
      method getNimbusConf (line 431) | public String getNimbusConf() throws org.apache.thrift7.TException
      method send_getNimbusConf (line 437) | public void send_getNimbusConf() throws org.apache.thrift7.TException
      method recv_getNimbusConf (line 443) | public String recv_getNimbusConf() throws org.apache.thrift7.TException
      method getClusterInfo (line 453) | public ClusterSummary getClusterInfo() throws org.apache.thrift7.TEx...
      method send_getClusterInfo (line 459) | public void send_getClusterInfo() throws org.apache.thrift7.TException
      method recv_getClusterInfo (line 465) | public ClusterSummary recv_getClusterInfo() throws org.apache.thrift...
      method getTopologyInfo (line 475) | public TopologyInfo getTopologyInfo(String id) throws NotAliveExcept...
      method send_getTopologyInfo (line 481) | public void send_getTopologyInfo(String id) throws org.apache.thrift...
      method recv_getTopologyInfo (line 488) | public TopologyInfo recv_getTopologyInfo() throws NotAliveException,...
      method getTopologyConf (line 501) | public String getTopologyConf(String id) throws NotAliveException, o...
      method send_getTopologyConf (line 507) | public void send_getTopologyConf(String id) throws org.apache.thrift...
      method recv_getTopologyConf (line 514) | public String recv_getTopologyConf() throws NotAliveException, org.a...
      method getTopology (line 527) | public StormTopology getTopology(String id) throws NotAliveException...
      method send_getTopology (line 533) | public void send_getTopology(String id) throws org.apache.thrift7.TE...
      method recv_getTopology (line 540) | public StormTopology recv_getTopology() throws NotAliveException, or...
      method getUserTopology (line 553) | public StormTopology getUserTopology(String id) throws NotAliveExcep...
      method send_getUserTopology (line 559) | public void send_getUserTopology(String id) throws org.apache.thrift...
      method recv_getUserTopology (line 566) | public StormTopology recv_getUserTopology() throws NotAliveException...
    class AsyncClient (line 580) | public static class AsyncClient extends org.apache.thrift7.async.TAsyn...
      class Factory (line 581) | public static class Factory implements org.apache.thrift7.async.TAsy...
        method Factory (line 584) | public Factory(org.apache.thrift7.async.TAsyncClientManager client...
        method getAsyncClient (line 588) | public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNo...
      method AsyncClient (line 593) | public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory prot...
      method submitTopology (line 597) | public void submitTopology(String name, String uploadedJarLocation, ...
      class submitTopology_call (line 604) | public static class submitTopology_call extends org.apache.thrift7.a...
        method submitTopology_call (line 609) | public submitTopology_call(String name, String uploadedJarLocation...
        method write_args (line 617) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 628) | public void getResult() throws AlreadyAliveException, InvalidTopol...
      method submitTopologyWithOpts (line 638) | public void submitTopologyWithOpts(String name, String uploadedJarLo...
      class submitTopologyWithOpts_call (line 645) | public static class submitTopologyWithOpts_call extends org.apache.t...
        method submitTopologyWithOpts_call (line 651) | public submitTopologyWithOpts_call(String name, String uploadedJar...
        method write_args (line 660) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 672) | public void getResult() throws AlreadyAliveException, InvalidTopol...
      method killTopology (line 682) | public void killTopology(String name, org.apache.thrift7.async.Async...
      class killTopology_call (line 689) | public static class killTopology_call extends org.apache.thrift7.asy...
        method killTopology_call (line 691) | public killTopology_call(String name, org.apache.thrift7.async.Asy...
        method write_args (line 696) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 704) | public void getResult() throws NotAliveException, org.apache.thrif...
      method killTopologyWithOpts (line 714) | public void killTopologyWithOpts(String name, KillOptions options, o...
      class killTopologyWithOpts_call (line 721) | public static class killTopologyWithOpts_call extends org.apache.thr...
        method killTopologyWithOpts_call (line 724) | public killTopologyWithOpts_call(String name, KillOptions options,...
        method write_args (line 730) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 739) | public void getResult() throws NotAliveException, org.apache.thrif...
      method activate (line 749) | public void activate(String name, org.apache.thrift7.async.AsyncMeth...
      class activate_call (line 756) | public static class activate_call extends org.apache.thrift7.async.T...
        method activate_call (line 758) | public activate_call(String name, org.apache.thrift7.async.AsyncMe...
        method write_args (line 763) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 771) | public void getResult() throws NotAliveException, org.apache.thrif...
      method deactivate (line 781) | public void deactivate(String name, org.apache.thrift7.async.AsyncMe...
      class deactivate_call (line 788) | public static class deactivate_call extends org.apache.thrift7.async...
        method deactivate_call (line 790) | public deactivate_call(String name, org.apache.thrift7.async.Async...
        method write_args (line 795) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 803) | public void getResult() throws NotAliveException, org.apache.thrif...
      method rebalance (line 813) | public void rebalance(String name, RebalanceOptions options, org.apa...
      class rebalance_call (line 820) | public static class rebalance_call extends org.apache.thrift7.async....
        method rebalance_call (line 823) | public rebalance_call(String name, RebalanceOptions options, org.a...
        method write_args (line 829) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 838) | public void getResult() throws NotAliveException, InvalidTopologyE...
      method beginFileUpload (line 848) | public void beginFileUpload(org.apache.thrift7.async.AsyncMethodCall...
      class beginFileUpload_call (line 855) | public static class beginFileUpload_call extends org.apache.thrift7....
        method beginFileUpload_call (line 856) | public beginFileUpload_call(org.apache.thrift7.async.AsyncMethodCa...
        method write_args (line 860) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 867) | public String getResult() throws org.apache.thrift7.TException {
      method uploadChunk (line 877) | public void uploadChunk(String location, ByteBuffer chunk, org.apach...
      class uploadChunk_call (line 884) | public static class uploadChunk_call extends org.apache.thrift7.asyn...
        method uploadChunk_call (line 887) | public uploadChunk_call(String location, ByteBuffer chunk, org.apa...
        method write_args (line 893) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 902) | public void getResult() throws org.apache.thrift7.TException {
      method finishFileUpload (line 912) | public void finishFileUpload(String location, org.apache.thrift7.asy...
      class finishFileUpload_call (line 919) | public static class finishFileUpload_call extends org.apache.thrift7...
        method finishFileUpload_call (line 921) | public finishFileUpload_call(String location, org.apache.thrift7.a...
        method write_args (line 926) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 934) | public void getResult() throws org.apache.thrift7.TException {
      method beginFileDownload (line 944) | public void beginFileDownload(String file, org.apache.thrift7.async....
      class beginFileDownload_call (line 951) | public static class beginFileDownload_call extends org.apache.thrift...
        method beginFileDownload_call (line 953) | public beginFileDownload_call(String file, org.apache.thrift7.asyn...
        method write_args (line 958) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 966) | public String getResult() throws org.apache.thrift7.TException {
      method downloadChunk (line 976) | public void downloadChunk(String id, org.apache.thrift7.async.AsyncM...
      class downloadChunk_call (line 983) | public static class downloadChunk_call extends org.apache.thrift7.as...
        method downloadChunk_call (line 985) | public downloadChunk_call(String id, org.apache.thrift7.async.Asyn...
        method write_args (line 990) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 998) | public ByteBuffer getResult() throws org.apache.thrift7.TException {
      method getNimbusConf (line 1008) | public void getNimbusConf(org.apache.thrift7.async.AsyncMethodCallba...
      class getNimbusConf_call (line 1015) | public static class getNimbusConf_call extends org.apache.thrift7.as...
        method getNimbusConf_call (line 1016) | public getNimbusConf_call(org.apache.thrift7.async.AsyncMethodCall...
        method write_args (line 1020) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 1027) | public String getResult() throws org.apache.thrift7.TException {
      method getClusterInfo (line 1037) | public void getClusterInfo(org.apache.thrift7.async.AsyncMethodCallb...
      class getClusterInfo_call (line 1044) | public static class getClusterInfo_call extends org.apache.thrift7.a...
        method getClusterInfo_call (line 1045) | public getClusterInfo_call(org.apache.thrift7.async.AsyncMethodCal...
        method write_args (line 1049) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 1056) | public ClusterSummary getResult() throws org.apache.thrift7.TExcep...
      method getTopologyInfo (line 1066) | public void getTopologyInfo(String id, org.apache.thrift7.async.Asyn...
      class getTopologyInfo_call (line 1073) | public static class getTopologyInfo_call extends org.apache.thrift7....
        method getTopologyInfo_call (line 1075) | public getTopologyInfo_call(String id, org.apache.thrift7.async.As...
        method write_args (line 1080) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 1088) | public TopologyInfo getResult() throws NotAliveException, org.apac...
      method getTopologyConf (line 1098) | public void getTopologyConf(String id, org.apache.thrift7.async.Asyn...
      class getTopologyConf_call (line 1105) | public static class getTopologyConf_call extends org.apache.thrift7....
        method getTopologyConf_call (line 1107) | public getTopologyConf_call(String id, org.apache.thrift7.async.As...
        method write_args (line 1112) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 1120) | public String getResult() throws NotAliveException, org.apache.thr...
      method getTopology (line 1130) | public void getTopology(String id, org.apache.thrift7.async.AsyncMet...
      class getTopology_call (line 1137) | public static class getTopology_call extends org.apache.thrift7.asyn...
        method getTopology_call (line 1139) | public getTopology_call(String id, org.apache.thrift7.async.AsyncM...
        method write_args (line 1144) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 1152) | public StormTopology getResult() throws NotAliveException, org.apa...
      method getUserTopology (line 1162) | public void getUserTopology(String id, org.apache.thrift7.async.Asyn...
      class getUserTopology_call (line 1169) | public static class getUserTopology_call extends org.apache.thrift7....
        method getUserTopology_call (line 1171) | public getUserTopology_call(String id, org.apache.thrift7.async.As...
        method write_args (line 1176) | public void write_args(org.apache.thrift7.protocol.TProtocol prot)...
        method getResult (line 1184) | public StormTopology getResult() throws NotAliveException, org.apa...
    class Processor (line 1196) | public static class Processor<I extends Iface> extends org.apache.thri...
      method Processor (line 1198) | public Processor(I iface) {
      method Processor (line 1202) | protected Processor(I iface, Map<String,  org.apache.thrift7.Process...
      method getProcessMap (line 1206) | private static <I extends Iface> Map<String,  org.apache.thrift7.Pro...
      class submitTopology (line 1228) | private static class submitTopology<I extends Iface> extends org.apa...
        method submitTopology (line 1229) | public submitTopology() {
        method getEmptyArgsInstance (line 1233) | protected submitTopology_args getEmptyArgsInstance() {
        method getResult (line 1237) | protected submitTopology_result getResult(I iface, submitTopology_...
      class submitTopologyWithOpts (line 1250) | private static class submitTopologyWithOpts<I extends Iface> extends...
        method submitTopologyWithOpts (line 1251) | public submitTopologyWithOpts() {
        method getEmptyArgsInstance (line 1255) | protected submitTopologyWithOpts_args getEmptyArgsInstance() {
        method getResult (line 1259) | protected submitTopologyWithOpts_result getResult(I iface, submitT...
      class killTopology (line 1272) | private static class killTopology<I extends Iface> extends org.apach...
        method killTopology (line 1273) | public killTopology() {
        method getEmptyArgsInstance (line 1277) | protected killTopology_args getEmptyArgsInstance() {
        method getResult (line 1281) | protected killTopology_result getResult(I iface, killTopology_args...
      class killTopologyWithOpts (line 1292) | private static class killTopologyWithOpts<I extends Iface> extends o...
        method killTopologyWithOpts (line 1293) | public killTopologyWithOpts() {
        method getEmptyArgsInstance (line 1297) | protected killTopologyWithOpts_args getEmptyArgsInstance() {
        method getResult (line 1301) | protected killTopologyWithOpts_result getResult(I iface, killTopol...
      class activate (line 1312) | private static class activate<I extends Iface> extends org.apache.th...
        method activate (line 1313) | public activate() {
        method getEmptyArgsInstance (line 1317) | protected activate_args getEmptyArgsInstance() {
        method getResult (line 1321) | protected activate_result getResult(I iface, activate_args args) t...
      class deactivate (line 1332) | private static class deactivate<I extends Iface> extends org.apache....
        method deactivate (line 1333) | public deactivate() {
        method getEmptyArgsInstance (line 1337) | protected deactivate_args getEmptyArgsInstance() {
        method getResult (line 1341) | protected deactivate_result getResult(I iface, deactivate_args arg...
      class rebalance (line 1352) | private static class rebalance<I extends Iface> extends org.apache.t...
        method rebalance (line 1353) | public rebalance() {
        method getEmptyArgsInstance (line 1357) | protected rebalance_args getEmptyArgsInstance() {
        method getResult (line 1361) | protected rebalance_result getResult(I iface, rebalance_args args)...
      class beginFileUpload (line 1374) | private static class beginFileUpload<I extends Iface> extends org.ap...
        method beginFileUpload (line 1375) | public beginFileUpload() {
        method getEmptyArgsInstance (line 1379) | protected beginFileUpload_args getEmptyArgsInstance() {
        method getResult (line 1383) | protected beginFileUpload_result getResult(I iface, beginFileUploa...
      class uploadChunk (line 1390) | private static class uploadChunk<I extends Iface> extends org.apache...
        method uploadChunk (line 1391) | public uploadChunk() {
        method getEmptyArgsInstance (line 1395) | protected uploadChunk_args getEmptyArgsInstance() {
        method getResult (line 1399) | protected uploadChunk_result getResult(I iface, uploadChunk_args a...
      class finishFileUpload (line 1406) | private static class finishFileUpload<I extends Iface> extends org.a...
        method finishFileUpload (line 1407) | public finishFileUpload() {
        method getEmptyArgsInstance (line 1411) | protected finishFileUpload_args getEmptyArgsInstance() {
        method getResult (line 1415) | protected finishFileUpload_result getResult(I iface, finishFileUpl...
      class beginFileDownload (line 1422) | private static class beginFileDownload<I extends Iface> extends org....
        method beginFileDownload (line 1423) | public beginFileDownload() {
        method getEmptyArgsInstance (line 1427) | protected beginFileDownload_args getEmptyArgsInstance() {
        method getResult (line 1431) | protected beginFileDownload_result getResult(I iface, beginFileDow...
      class downloadChunk (line 1438) | private static class downloadChunk<I extends Iface> extends org.apac...
        method downloadChunk (line 1439) | public downloadChunk() {
        method getEmptyArgsInstance (line 1443) | protected downloadChunk_args getEmptyArgsInstance() {
        method getResult (line 1447) | protected downloadChunk_result getResult(I iface, downloadChunk_ar...
      class getNimbusConf (line 1454) | private static class getNimbusConf<I extends Iface> extends org.apac...
        method getNimbusConf (line 1455) | public getNimbusConf() {
        method getEmptyArgsInstance (line 1459) | protected getNimbusConf_args getEmptyArgsInstance() {
        method getResult (line 1463) | protected getNimbusConf_result getResult(I iface, getNimbusConf_ar...
      class getClusterInfo (line 1470) | private static class getClusterInfo<I extends Iface> extends org.apa...
        method getClusterInfo (line 1471) | public getClusterInfo() {
        method getEmptyArgsInstance (line 1475) | protected getClusterInfo_args getEmptyArgsInstance() {
        method getResult (line 1479) | protected getClusterInfo_result getResult(I iface, getClusterInfo_...
      class getTopologyInfo (line 1486) | private static class getTopologyInfo<I extends Iface> extends org.ap...
        method getTopologyInfo (line 1487) | public getTopologyInfo() {
        method getEmptyArgsInstance (line 1491) | protected getTopologyInfo_args getEmptyArgsInstance() {
        method getResult (line 1495) | protected getTopologyInfo_result getResult(I iface, getTopologyInf...
      class getTopologyConf (line 1506) | private static class getTopologyConf<I extends Iface> extends org.ap...
        method getTopologyConf (line 1507) | public getTopologyConf() {
        method getEmptyArgsInstance (line 1511) | protected getTopologyConf_args getEmptyArgsInstance() {
        method getResult (line 1515) | protected getTopologyConf_result getResult(I iface, getTopologyCon...
      class getTopology (line 1526) | private static class getTopology<I extends Iface> extends org.apache...
        method getTopology (line 1527) | public getTopology() {
        method getEmptyArgsInstance (line 1531) | protected getTopology_args getEmptyArgsInstance() {
        method getResult (line 1535) | protected getTopology_result getResult(I iface, getTopology_args a...
      class getUserTopology (line 1546) | private static class getUserTopology<I extends Iface> extends org.ap...
        method getUserTopology (line 1547) | public getUserTopology() {
        method getEmptyArgsInstance (line 1551) | protected getUserTopology_args getEmptyArgsInstance() {
        method getResult (line 1555) | protected getUserTopology_result getResult(I iface, getUserTopolog...
    class submitTopology_args (line 1568) | public static class submitTopology_args implements org.apache.thrift7....
      type _Fields (line 1582) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 1599) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 1618) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 1627) | public static _Fields findByName(String name) {
        method _Fields (line 1634) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 1639) | public short getThriftFieldId() {
        method getFieldName (line 1643) | public String getFieldName() {
      method submitTopology_args (line 1665) | public submitTopology_args() {
      method submitTopology_args (line 1668) | public submitTopology_args(
      method submitTopology_args (line 1684) | public submitTopology_args(submitTopology_args other) {
      method deepCopy (line 1699) | public submitTopology_args deepCopy() {
      method clear (line 1703) | @Override
      method get_name (line 1711) | public String get_name() {
      method set_name (line 1715) | public void set_name(String name) {
      method unset_name (line 1719) | public void unset_name() {
      method is_set_name (line 1724) | public boolean is_set_name() {
      method set_name_isSet (line 1728) | public void set_name_isSet(boolean value) {
      method get_uploadedJarLocation (line 1734) | public String get_uploadedJarLocation() {
      method set_uploadedJarLocation (line 1738) | public void set_uploadedJarLocation(String uploadedJarLocation) {
      method unset_uploadedJarLocation (line 1742) | public void unset_uploadedJarLocation() {
      method is_set_uploadedJarLocation (line 1747) | public boolean is_set_uploadedJarLocation() {
      method set_uploadedJarLocation_isSet (line 1751) | public void set_uploadedJarLocation_isSet(boolean value) {
      method get_jsonConf (line 1757) | public String get_jsonConf() {
      method set_jsonConf (line 1761) | public void set_jsonConf(String jsonConf) {
      method unset_jsonConf (line 1765) | public void unset_jsonConf() {
      method is_set_jsonConf (line 1770) | public boolean is_set_jsonConf() {
      method set_jsonConf_isSet (line 1774) | public void set_jsonConf_isSet(boolean value) {
      method get_topology (line 1780) | public StormTopology get_topology() {
      method set_topology (line 1784) | public void set_topology(StormTopology topology) {
      method unset_topology (line 1788) | public void unset_topology() {
      method is_set_topology (line 1793) | public boolean is_set_topology() {
      method set_topology_isSet (line 1797) | public void set_topology_isSet(boolean value) {
      method setFieldValue (line 1803) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 1840) | public Object getFieldValue(_Fields field) {
      method isSet (line 1859) | public boolean isSet(_Fields field) {
      method equals (line 1877) | @Override
      method equals (line 1886) | public boolean equals(submitTopology_args that) {
      method hashCode (line 1929) | @Override
      method compareTo (line 1956) | public int compareTo(submitTopology_args other) {
      method fieldForId (line 2007) | public _Fields fieldForId(int fieldId) {
      method read (line 2011) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 2059) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 2087) | @Override
      method validate (line 2127) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 2131) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 2139) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class submitTopology_result (line 2149) | public static class submitTopology_result implements org.apache.thrift...
      type _Fields (line 2159) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 2174) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 2189) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 2198) | public static _Fields findByName(String name) {
        method _Fields (line 2205) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 2210) | public short getThriftFieldId() {
        method getFieldName (line 2214) | public String getFieldName() {
      method submitTopology_result (line 2232) | public submitTopology_result() {
      method submitTopology_result (line 2235) | public submitTopology_result(
      method submitTopology_result (line 2247) | public submitTopology_result(submitTopology_result other) {
      method deepCopy (line 2256) | public submitTopology_result deepCopy() {
      method clear (line 2260) | @Override
      method get_e (line 2266) | public AlreadyAliveException get_e() {
      method set_e (line 2270) | public void set_e(AlreadyAliveException e) {
      method unset_e (line 2274) | public void unset_e() {
      method is_set_e (line 2279) | public boolean is_set_e() {
      method set_e_isSet (line 2283) | public void set_e_isSet(boolean value) {
      method get_ite (line 2289) | public InvalidTopologyException get_ite() {
      method set_ite (line 2293) | public void set_ite(InvalidTopologyException ite) {
      method unset_ite (line 2297) | public void unset_ite() {
      method is_set_ite (line 2302) | public boolean is_set_ite() {
      method set_ite_isSet (line 2306) | public void set_ite_isSet(boolean value) {
      method setFieldValue (line 2312) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 2333) | public Object getFieldValue(_Fields field) {
      method isSet (line 2346) | public boolean isSet(_Fields field) {
      method equals (line 2360) | @Override
      method equals (line 2369) | public boolean equals(submitTopology_result that) {
      method hashCode (line 2394) | @Override
      method compareTo (line 2411) | public int compareTo(submitTopology_result other) {
      method fieldForId (line 2442) | public _Fields fieldForId(int fieldId) {
      method read (line 2446) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 2481) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 2497) | @Override
      method validate (line 2521) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 2525) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 2533) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class submitTopologyWithOpts_args (line 2543) | public static class submitTopologyWithOpts_args implements org.apache....
      type _Fields (line 2559) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 2577) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 2598) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 2607) | public static _Fields findByName(String name) {
        method _Fields (line 2614) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 2619) | public short getThriftFieldId() {
        method getFieldName (line 2623) | public String getFieldName() {
      method submitTopologyWithOpts_args (line 2647) | public submitTopologyWithOpts_args() {
      method submitTopologyWithOpts_args (line 2650) | public submitTopologyWithOpts_args(
      method submitTopologyWithOpts_args (line 2668) | public submitTopologyWithOpts_args(submitTopologyWithOpts_args other) {
      method deepCopy (line 2686) | public submitTopologyWithOpts_args deepCopy() {
      method clear (line 2690) | @Override
      method get_name (line 2699) | public String get_name() {
      method set_name (line 2703) | public void set_name(String name) {
      method unset_name (line 2707) | public void unset_name() {
      method is_set_name (line 2712) | public boolean is_set_name() {
      method set_name_isSet (line 2716) | public void set_name_isSet(boolean value) {
      method get_uploadedJarLocation (line 2722) | public String get_uploadedJarLocation() {
      method set_uploadedJarLocation (line 2726) | public void set_uploadedJarLocation(String uploadedJarLocation) {
      method unset_uploadedJarLocation (line 2730) | public void unset_uploadedJarLocation() {
      method is_set_uploadedJarLocation (line 2735) | public boolean is_set_uploadedJarLocation() {
      method set_uploadedJarLocation_isSet (line 2739) | public void set_uploadedJarLocation_isSet(boolean value) {
      method get_jsonConf (line 2745) | public String get_jsonConf() {
      method set_jsonConf (line 2749) | public void set_jsonConf(String jsonConf) {
      method unset_jsonConf (line 2753) | public void unset_jsonConf() {
      method is_set_jsonConf (line 2758) | public boolean is_set_jsonConf() {
      method set_jsonConf_isSet (line 2762) | public void set_jsonConf_isSet(boolean value) {
      method get_topology (line 2768) | public StormTopology get_topology() {
      method set_topology (line 2772) | public void set_topology(StormTopology topology) {
      method unset_topology (line 2776) | public void unset_topology() {
      method is_set_topology (line 2781) | public boolean is_set_topology() {
      method set_topology_isSet (line 2785) | public void set_topology_isSet(boolean value) {
      method get_options (line 2791) | public SubmitOptions get_options() {
      method set_options (line 2795) | public void set_options(SubmitOptions options) {
      method unset_options (line 2799) | public void unset_options() {
      method is_set_options (line 2804) | public boolean is_set_options() {
      method set_options_isSet (line 2808) | public void set_options_isSet(boolean value) {
      method setFieldValue (line 2814) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 2859) | public Object getFieldValue(_Fields field) {
      method isSet (line 2881) | public boolean isSet(_Fields field) {
      method equals (line 2901) | @Override
      method equals (line 2910) | public boolean equals(submitTopologyWithOpts_args that) {
      method hashCode (line 2962) | @Override
      method compareTo (line 2994) | public int compareTo(submitTopologyWithOpts_args other) {
      method fieldForId (line 3055) | public _Fields fieldForId(int fieldId) {
      method read (line 3059) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 3115) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 3148) | @Override
      method validate (line 3196) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 3200) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 3208) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class submitTopologyWithOpts_result (line 3218) | public static class submitTopologyWithOpts_result implements org.apach...
      type _Fields (line 3228) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 3243) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 3258) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 3267) | public static _Fields findByName(String name) {
        method _Fields (line 3274) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 3279) | public short getThriftFieldId() {
        method getFieldName (line 3283) | public String getFieldName() {
      method submitTopologyWithOpts_result (line 3301) | public submitTopologyWithOpts_result() {
      method submitTopologyWithOpts_result (line 3304) | public submitTopologyWithOpts_result(
      method submitTopologyWithOpts_result (line 3316) | public submitTopologyWithOpts_result(submitTopologyWithOpts_result o...
      method deepCopy (line 3325) | public submitTopologyWithOpts_result deepCopy() {
      method clear (line 3329) | @Override
      method get_e (line 3335) | public AlreadyAliveException get_e() {
      method set_e (line 3339) | public void set_e(AlreadyAliveException e) {
      method unset_e (line 3343) | public void unset_e() {
      method is_set_e (line 3348) | public boolean is_set_e() {
      method set_e_isSet (line 3352) | public void set_e_isSet(boolean value) {
      method get_ite (line 3358) | public InvalidTopologyException get_ite() {
      method set_ite (line 3362) | public void set_ite(InvalidTopologyException ite) {
      method unset_ite (line 3366) | public void unset_ite() {
      method is_set_ite (line 3371) | public boolean is_set_ite() {
      method set_ite_isSet (line 3375) | public void set_ite_isSet(boolean value) {
      method setFieldValue (line 3381) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 3402) | public Object getFieldValue(_Fields field) {
      method isSet (line 3415) | public boolean isSet(_Fields field) {
      method equals (line 3429) | @Override
      method equals (line 3438) | public boolean equals(submitTopologyWithOpts_result that) {
      method hashCode (line 3463) | @Override
      method compareTo (line 3480) | public int compareTo(submitTopologyWithOpts_result other) {
      method fieldForId (line 3511) | public _Fields fieldForId(int fieldId) {
      method read (line 3515) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 3550) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 3566) | @Override
      method validate (line 3590) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 3594) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 3602) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class killTopology_args (line 3612) | public static class killTopology_args implements org.apache.thrift7.TB...
      type _Fields (line 3620) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 3634) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 3647) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 3656) | public static _Fields findByName(String name) {
        method _Fields (line 3663) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 3668) | public short getThriftFieldId() {
        method getFieldName (line 3672) | public String getFieldName() {
      method killTopology_args (line 3688) | public killTopology_args() {
      method killTopology_args (line 3691) | public killTopology_args(
      method killTopology_args (line 3701) | public killTopology_args(killTopology_args other) {
      method deepCopy (line 3707) | public killTopology_args deepCopy() {
      method clear (line 3711) | @Override
      method get_name (line 3716) | public String get_name() {
      method set_name (line 3720) | public void set_name(String name) {
      method unset_name (line 3724) | public void unset_name() {
      method is_set_name (line 3729) | public boolean is_set_name() {
      method set_name_isSet (line 3733) | public void set_name_isSet(boolean value) {
      method setFieldValue (line 3739) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 3752) | public Object getFieldValue(_Fields field) {
      method isSet (line 3762) | public boolean isSet(_Fields field) {
      method equals (line 3774) | @Override
      method equals (line 3783) | public boolean equals(killTopology_args that) {
      method hashCode (line 3799) | @Override
      method compareTo (line 3811) | public int compareTo(killTopology_args other) {
      method fieldForId (line 3832) | public _Fields fieldForId(int fieldId) {
      method read (line 3836) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 3862) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 3875) | @Override
      method validate (line 3891) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 3895) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 3903) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class killTopology_result (line 3913) | public static class killTopology_result implements org.apache.thrift7....
      type _Fields (line 3921) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 3935) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 3948) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 3957) | public static _Fields findByName(String name) {
        method _Fields (line 3964) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 3969) | public short getThriftFieldId() {
        method getFieldName (line 3973) | public String getFieldName() {
      method killTopology_result (line 3989) | public killTopology_result() {
      method killTopology_result (line 3992) | public killTopology_result(
      method killTopology_result (line 4002) | public killTopology_result(killTopology_result other) {
      method deepCopy (line 4008) | public killTopology_result deepCopy() {
      method clear (line 4012) | @Override
      method get_e (line 4017) | public NotAliveException get_e() {
      method set_e (line 4021) | public void set_e(NotAliveException e) {
      method unset_e (line 4025) | public void unset_e() {
      method is_set_e (line 4030) | public boolean is_set_e() {
      method set_e_isSet (line 4034) | public void set_e_isSet(boolean value) {
      method setFieldValue (line 4040) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 4053) | public Object getFieldValue(_Fields field) {
      method isSet (line 4063) | public boolean isSet(_Fields field) {
      method equals (line 4075) | @Override
      method equals (line 4084) | public boolean equals(killTopology_result that) {
      method hashCode (line 4100) | @Override
      method compareTo (line 4112) | public int compareTo(killTopology_result other) {
      method fieldForId (line 4133) | public _Fields fieldForId(int fieldId) {
      method read (line 4137) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 4164) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 4176) | @Override
      method validate (line 4192) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 4196) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 4204) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class killTopologyWithOpts_args (line 4214) | public static class killTopologyWithOpts_args implements org.apache.th...
      type _Fields (line 4224) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 4239) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 4254) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 4263) | public static _Fields findByName(String name) {
        method _Fields (line 4270) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 4275) | public short getThriftFieldId() {
        method getFieldName (line 4279) | public String getFieldName() {
      method killTopologyWithOpts_args (line 4297) | public killTopologyWithOpts_args() {
      method killTopologyWithOpts_args (line 4300) | public killTopologyWithOpts_args(
      method killTopologyWithOpts_args (line 4312) | public killTopologyWithOpts_args(killTopologyWithOpts_args other) {
      method deepCopy (line 4321) | public killTopologyWithOpts_args deepCopy() {
      method clear (line 4325) | @Override
      method get_name (line 4331) | public String get_name() {
      method set_name (line 4335) | public void set_name(String name) {
      method unset_name (line 4339) | public void unset_name() {
      method is_set_name (line 4344) | public boolean is_set_name() {
      method set_name_isSet (line 4348) | public void set_name_isSet(boolean value) {
      method get_options (line 4354) | public KillOptions get_options() {
      method set_options (line 4358) | public void set_options(KillOptions options) {
      method unset_options (line 4362) | public void unset_options() {
      method is_set_options (line 4367) | public boolean is_set_options() {
      method set_options_isSet (line 4371) | public void set_options_isSet(boolean value) {
      method setFieldValue (line 4377) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 4398) | public Object getFieldValue(_Fields field) {
      method isSet (line 4411) | public boolean isSet(_Fields field) {
      method equals (line 4425) | @Override
      method equals (line 4434) | public boolean equals(killTopologyWithOpts_args that) {
      method hashCode (line 4459) | @Override
      method compareTo (line 4476) | public int compareTo(killTopologyWithOpts_args other) {
      method fieldForId (line 4507) | public _Fields fieldForId(int fieldId) {
      method read (line 4511) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 4545) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 4563) | @Override
      method validate (line 4587) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 4591) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 4599) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class killTopologyWithOpts_result (line 4609) | public static class killTopologyWithOpts_result implements org.apache....
      type _Fields (line 4617) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 4631) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 4644) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 4653) | public static _Fields findByName(String name) {
        method _Fields (line 4660) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 4665) | public short getThriftFieldId() {
        method getFieldName (line 4669) | public String getFieldName() {
      method killTopologyWithOpts_result (line 4685) | public killTopologyWithOpts_result() {
      method killTopologyWithOpts_result (line 4688) | public killTopologyWithOpts_result(
      method killTopologyWithOpts_result (line 4698) | public killTopologyWithOpts_result(killTopologyWithOpts_result other) {
      method deepCopy (line 4704) | public killTopologyWithOpts_result deepCopy() {
      method clear (line 4708) | @Override
      method get_e (line 4713) | public NotAliveException get_e() {
      method set_e (line 4717) | public void set_e(NotAliveException e) {
      method unset_e (line 4721) | public void unset_e() {
      method is_set_e (line 4726) | public boolean is_set_e() {
      method set_e_isSet (line 4730) | public void set_e_isSet(boolean value) {
      method setFieldValue (line 4736) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 4749) | public Object getFieldValue(_Fields field) {
      method isSet (line 4759) | public boolean isSet(_Fields field) {
      method equals (line 4771) | @Override
      method equals (line 4780) | public boolean equals(killTopologyWithOpts_result that) {
      method hashCode (line 4796) | @Override
      method compareTo (line 4808) | public int compareTo(killTopologyWithOpts_result other) {
      method fieldForId (line 4829) | public _Fields fieldForId(int fieldId) {
      method read (line 4833) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 4860) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 4872) | @Override
      method validate (line 4888) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 4892) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 4900) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class activate_args (line 4910) | public static class activate_args implements org.apache.thrift7.TBase<...
      type _Fields (line 4918) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 4932) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 4945) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 4954) | public static _Fields findByName(String name) {
        method _Fields (line 4961) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 4966) | public short getThriftFieldId() {
        method getFieldName (line 4970) | public String getFieldName() {
      method activate_args (line 4986) | public activate_args() {
      method activate_args (line 4989) | public activate_args(
      method activate_args (line 4999) | public activate_args(activate_args other) {
      method deepCopy (line 5005) | public activate_args deepCopy() {
      method clear (line 5009) | @Override
      method get_name (line 5014) | public String get_name() {
      method set_name (line 5018) | public void set_name(String name) {
      method unset_name (line 5022) | public void unset_name() {
      method is_set_name (line 5027) | public boolean is_set_name() {
      method set_name_isSet (line 5031) | public void set_name_isSet(boolean value) {
      method setFieldValue (line 5037) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 5050) | public Object getFieldValue(_Fields field) {
      method isSet (line 5060) | public boolean isSet(_Fields field) {
      method equals (line 5072) | @Override
      method equals (line 5081) | public boolean equals(activate_args that) {
      method hashCode (line 5097) | @Override
      method compareTo (line 5109) | public int compareTo(activate_args other) {
      method fieldForId (line 5130) | public _Fields fieldForId(int fieldId) {
      method read (line 5134) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 5160) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 5173) | @Override
      method validate (line 5189) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 5193) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 5201) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class activate_result (line 5211) | public static class activate_result implements org.apache.thrift7.TBas...
      type _Fields (line 5219) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 5233) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 5246) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 5255) | public static _Fields findByName(String name) {
        method _Fields (line 5262) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 5267) | public short getThriftFieldId() {
        method getFieldName (line 5271) | public String getFieldName() {
      method activate_result (line 5287) | public activate_result() {
      method activate_result (line 5290) | public activate_result(
      method activate_result (line 5300) | public activate_result(activate_result other) {
      method deepCopy (line 5306) | public activate_result deepCopy() {
      method clear (line 5310) | @Override
      method get_e (line 5315) | public NotAliveException get_e() {
      method set_e (line 5319) | public void set_e(NotAliveException e) {
      method unset_e (line 5323) | public void unset_e() {
      method is_set_e (line 5328) | public boolean is_set_e() {
      method set_e_isSet (line 5332) | public void set_e_isSet(boolean value) {
      method setFieldValue (line 5338) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 5351) | public Object getFieldValue(_Fields field) {
      method isSet (line 5361) | public boolean isSet(_Fields field) {
      method equals (line 5373) | @Override
      method equals (line 5382) | public boolean equals(activate_result that) {
      method hashCode (line 5398) | @Override
      method compareTo (line 5410) | public int compareTo(activate_result other) {
      method fieldForId (line 5431) | public _Fields fieldForId(int fieldId) {
      method read (line 5435) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 5462) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 5474) | @Override
      method validate (line 5490) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 5494) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 5502) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class deactivate_args (line 5512) | public static class deactivate_args implements org.apache.thrift7.TBas...
      type _Fields (line 5520) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 5534) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 5547) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 5556) | public static _Fields findByName(String name) {
        method _Fields (line 5563) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 5568) | public short getThriftFieldId() {
        method getFieldName (line 5572) | public String getFieldName() {
      method deactivate_args (line 5588) | public deactivate_args() {
      method deactivate_args (line 5591) | public deactivate_args(
      method deactivate_args (line 5601) | public deactivate_args(deactivate_args other) {
      method deepCopy (line 5607) | public deactivate_args deepCopy() {
      method clear (line 5611) | @Override
      method get_name (line 5616) | public String get_name() {
      method set_name (line 5620) | public void set_name(String name) {
      method unset_name (line 5624) | public void unset_name() {
      method is_set_name (line 5629) | public boolean is_set_name() {
      method set_name_isSet (line 5633) | public void set_name_isSet(boolean value) {
      method setFieldValue (line 5639) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 5652) | public Object getFieldValue(_Fields field) {
      method isSet (line 5662) | public boolean isSet(_Fields field) {
      method equals (line 5674) | @Override
      method equals (line 5683) | public boolean equals(deactivate_args that) {
      method hashCode (line 5699) | @Override
      method compareTo (line 5711) | public int compareTo(deactivate_args other) {
      method fieldForId (line 5732) | public _Fields fieldForId(int fieldId) {
      method read (line 5736) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 5762) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 5775) | @Override
      method validate (line 5791) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 5795) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 5803) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class deactivate_result (line 5813) | public static class deactivate_result implements org.apache.thrift7.TB...
      type _Fields (line 5821) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 5835) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 5848) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 5857) | public static _Fields findByName(String name) {
        method _Fields (line 5864) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 5869) | public short getThriftFieldId() {
        method getFieldName (line 5873) | public String getFieldName() {
      method deactivate_result (line 5889) | public deactivate_result() {
      method deactivate_result (line 5892) | public deactivate_result(
      method deactivate_result (line 5902) | public deactivate_result(deactivate_result other) {
      method deepCopy (line 5908) | public deactivate_result deepCopy() {
      method clear (line 5912) | @Override
      method get_e (line 5917) | public NotAliveException get_e() {
      method set_e (line 5921) | public void set_e(NotAliveException e) {
      method unset_e (line 5925) | public void unset_e() {
      method is_set_e (line 5930) | public boolean is_set_e() {
      method set_e_isSet (line 5934) | public void set_e_isSet(boolean value) {
      method setFieldValue (line 5940) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 5953) | public Object getFieldValue(_Fields field) {
      method isSet (line 5963) | public boolean isSet(_Fields field) {
      method equals (line 5975) | @Override
      method equals (line 5984) | public boolean equals(deactivate_result that) {
      method hashCode (line 6000) | @Override
      method compareTo (line 6012) | public int compareTo(deactivate_result other) {
      method fieldForId (line 6033) | public _Fields fieldForId(int fieldId) {
      method read (line 6037) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 6064) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 6076) | @Override
      method validate (line 6092) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 6096) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 6104) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class rebalance_args (line 6114) | public static class rebalance_args implements org.apache.thrift7.TBase...
      type _Fields (line 6124) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 6139) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 6154) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 6163) | public static _Fields findByName(String name) {
        method _Fields (line 6170) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 6175) | public short getThriftFieldId() {
        method getFieldName (line 6179) | public String getFieldName() {
      method rebalance_args (line 6197) | public rebalance_args() {
      method rebalance_args (line 6200) | public rebalance_args(
      method rebalance_args (line 6212) | public rebalance_args(rebalance_args other) {
      method deepCopy (line 6221) | public rebalance_args deepCopy() {
      method clear (line 6225) | @Override
      method get_name (line 6231) | public String get_name() {
      method set_name (line 6235) | public void set_name(String name) {
      method unset_name (line 6239) | public void unset_name() {
      method is_set_name (line 6244) | public boolean is_set_name() {
      method set_name_isSet (line 6248) | public void set_name_isSet(boolean value) {
      method get_options (line 6254) | public RebalanceOptions get_options() {
      method set_options (line 6258) | public void set_options(RebalanceOptions options) {
      method unset_options (line 6262) | public void unset_options() {
      method is_set_options (line 6267) | public boolean is_set_options() {
      method set_options_isSet (line 6271) | public void set_options_isSet(boolean value) {
      method setFieldValue (line 6277) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 6298) | public Object getFieldValue(_Fields field) {
      method isSet (line 6311) | public boolean isSet(_Fields field) {
      method equals (line 6325) | @Override
      method equals (line 6334) | public boolean equals(rebalance_args that) {
      method hashCode (line 6359) | @Override
      method compareTo (line 6376) | public int compareTo(rebalance_args other) {
      method fieldForId (line 6407) | public _Fields fieldForId(int fieldId) {
      method read (line 6411) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 6445) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 6463) | @Override
      method validate (line 6487) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 6491) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 6499) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class rebalance_result (line 6509) | public static class rebalance_result implements org.apache.thrift7.TBa...
      type _Fields (line 6519) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 6534) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 6549) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 6558) | public static _Fields findByName(String name) {
        method _Fields (line 6565) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 6570) | public short getThriftFieldId() {
        method getFieldName (line 6574) | public String getFieldName() {
      method rebalance_result (line 6592) | public rebalance_result() {
      method rebalance_result (line 6595) | public rebalance_result(
      method rebalance_result (line 6607) | public rebalance_result(rebalance_result other) {
      method deepCopy (line 6616) | public rebalance_result deepCopy() {
      method clear (line 6620) | @Override
      method get_e (line 6626) | public NotAliveException get_e() {
      method set_e (line 6630) | public void set_e(NotAliveException e) {
      method unset_e (line 6634) | public void unset_e() {
      method is_set_e (line 6639) | public boolean is_set_e() {
      method set_e_isSet (line 6643) | public void set_e_isSet(boolean value) {
      method get_ite (line 6649) | public InvalidTopologyException get_ite() {
      method set_ite (line 6653) | public void set_ite(InvalidTopologyException ite) {
      method unset_ite (line 6657) | public void unset_ite() {
      method is_set_ite (line 6662) | public boolean is_set_ite() {
      method set_ite_isSet (line 6666) | public void set_ite_isSet(boolean value) {
      method setFieldValue (line 6672) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 6693) | public Object getFieldValue(_Fields field) {
      method isSet (line 6706) | public boolean isSet(_Fields field) {
      method equals (line 6720) | @Override
      method equals (line 6729) | public boolean equals(rebalance_result that) {
      method hashCode (line 6754) | @Override
      method compareTo (line 6771) | public int compareTo(rebalance_result other) {
      method fieldForId (line 6802) | public _Fields fieldForId(int fieldId) {
      method read (line 6806) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 6841) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 6857) | @Override
      method validate (line 6881) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 6885) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 6893) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class beginFileUpload_args (line 6903) | public static class beginFileUpload_args implements org.apache.thrift7...
      type _Fields (line 6909) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 6923) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 6934) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 6943) | public static _Fields findByName(String name) {
        method _Fields (line 6950) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 6955) | public short getThriftFieldId() {
        method getFieldName (line 6959) | public String getFieldName() {
      method beginFileUpload_args (line 6970) | public beginFileUpload_args() {
      method beginFileUpload_args (line 6976) | public beginFileUpload_args(beginFileUpload_args other) {
      method deepCopy (line 6979) | public beginFileUpload_args deepCopy() {
      method clear (line 6983) | @Override
      method setFieldValue (line 6987) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 6992) | public Object getFieldValue(_Fields field) {
      method isSet (line 6999) | public boolean isSet(_Fields field) {
      method equals (line 7009) | @Override
      method equals (line 7018) | public boolean equals(beginFileUpload_args that) {
      method hashCode (line 7025) | @Override
      method compareTo (line 7032) | public int compareTo(beginFileUpload_args other) {
      method fieldForId (line 7043) | public _Fields fieldForId(int fieldId) {
      method read (line 7047) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 7066) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 7074) | @Override
      method validate (line 7083) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 7087) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 7095) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class beginFileUpload_result (line 7105) | public static class beginFileUpload_result implements org.apache.thrif...
      type _Fields (line 7113) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 7127) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 7140) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 7149) | public static _Fields findByName(String name) {
        method _Fields (line 7156) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 7161) | public short getThriftFieldId() {
        method getFieldName (line 7165) | public String getFieldName() {
      method beginFileUpload_result (line 7181) | public beginFileUpload_result() {
      method beginFileUpload_result (line 7184) | public beginFileUpload_result(
      method beginFileUpload_result (line 7194) | public beginFileUpload_result(beginFileUpload_result other) {
      method deepCopy (line 7200) | public beginFileUpload_result deepCopy() {
      method clear (line 7204) | @Override
      method get_success (line 7209) | public String get_success() {
      method set_success (line 7213) | public void set_success(String success) {
      method unset_success (line 7217) | public void unset_success() {
      method is_set_success (line 7222) | public boolean is_set_success() {
      method set_success_isSet (line 7226) | public void set_success_isSet(boolean value) {
      method setFieldValue (line 7232) | public void setFieldValue(_Fields field, Object value) {
      method getFieldValue (line 7245) | public Object getFieldValue(_Fields field) {
      method isSet (line 7255) | public boolean isSet(_Fields field) {
      method equals (line 7267) | @Override
      method equals (line 7276) | public boolean equals(beginFileUpload_result that) {
      method hashCode (line 7292) | @Override
      method compareTo (line 7304) | public int compareTo(beginFileUpload_result other) {
      method fieldForId (line 7325) | public _Fields fieldForId(int fieldId) {
      method read (line 7329) | public void read(org.apache.thrift7.protocol.TProtocol iprot) throws...
      method write (line 7355) | public void write(org.apache.thrift7.protocol.TProtocol oprot) throw...
      method toString (line 7367) | @Override
      method validate (line 7383) | public void validate() throws org.apache.thrift7.TException {
      method writeObject (line 7387) | private void writeObject(java.io.ObjectOutputStream out) throws java...
      method readObject (line 7395) | private void readObject(java.io.ObjectInputStream in) throws java.io...
    class uploadChunk_args (line 7405) | public static class uploadChunk_args implements org.apache.thrift7.TBa...
      type _Fields (line 7415) | public enum _Fields implements org.apache.thrift7.TFieldIdEnum {
        method findByThriftId (line 7430) | public static _Fields findByThriftId(int fieldId) {
        method findByThriftIdOrThrow (line 7445) | public static _Fields findByThriftIdOrThrow(int fieldId) {
        method findByName (line 7454) | public static _Fields findByName(String name) {
        method _Fields (line 7461) | _Fields(short thriftId, String fieldName) {
        method getThriftFieldId (line 7466) | public short getThriftFieldId() {
        method getFieldName (line 7470) | public String getFieldName() {
      method uploadChunk_args (line 7488) | public uploadChunk_args() {
      method uploadChunk_args (line 7491) | public uploadChunk_args(
      method uploadChunk_args (line 7503) | public uploadChunk_args(uploadChunk_args other) {
      method deepCopy (line 7513) | public uploadChunk_args deepCopy() {
      method clear (line 7517) | @Override
      method get_location (line 7523) | public String get_location() {
      method set_location (line 7527) | public void set_location(String location) {
      method unset_location (line 7531) | public void unset_loc
Condensed preview — 585 files, each showing path, character count, and a content snippet. Download the .json file or copy for the full structured content (3,564K chars).
[
  {
    "path": ".gitignore",
    "chars": 303,
    "preview": ".lein-repl-history\n/classes\n/lib\ndeploy/lib\ndeploy/logs\n.emacs-project\n*.jar\nbin/jzmq\n.DS_Store\npom.xml\ndeploy/classes\n*"
  },
  {
    "path": "CHANGELOG.md",
    "chars": 29410,
    "preview": "## 0.9.0.1\n* Update build configuration to force compatibility with Java 1.6\n\n## 0.9.0\n* Fixed a netty client issue wher"
  },
  {
    "path": "KEYS",
    "chars": 1195,
    "preview": "pub   2048R/E80B8FFD 2012-03-05 [expires: 2018-03-05]\nuid                  P. Taylor Goetz <ptgoetz@gmail.com>\nsig 3    "
  },
  {
    "path": "LICENSE",
    "chars": 15255,
    "preview": "\n                                 Apache License\n                           Version 2.0, January 2004\n                  "
  },
  {
    "path": "MODULES",
    "chars": 46,
    "preview": "storm-console-logging\nstorm-core\nstorm-netty\n\n"
  },
  {
    "path": "NOTICE",
    "chars": 381,
    "preview": "Apache Storm\nCopyright 2013 The Apache Software Foundation\n\nThis product includes software developed at\nThe Apache Softw"
  },
  {
    "path": "README.markdown",
    "chars": 3023,
    "preview": "## IMPORTANT NOTE!!!\nStorm has Moved to Apache. The official Storm git repository is now hosted by Apache, and is mirror"
  },
  {
    "path": "TODO",
    "chars": 9959,
    "preview": "Use cases:\n\n1. number of steps between 2 people in a graph (topology with cycles?)\n\n\n#################\n\n* Repackage jzmq"
  },
  {
    "path": "VERSION",
    "chars": 26,
    "preview": "0.9.1-incubating-SNAPSHOT\n"
  },
  {
    "path": "bin/build_modules.sh",
    "chars": 1213,
    "preview": "#!/bin/bash\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See"
  },
  {
    "path": "bin/build_release.sh",
    "chars": 1787,
    "preview": "#!/bin/bash\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See"
  },
  {
    "path": "bin/install_zmq.sh",
    "chars": 1474,
    "preview": "#!/bin/bash\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See"
  },
  {
    "path": "bin/javadoc.sh",
    "chars": 857,
    "preview": "# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See the NOTICE f"
  },
  {
    "path": "bin/storm",
    "chars": 16275,
    "preview": "#!/usr/bin/python\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements"
  },
  {
    "path": "bin/to_maven.sh",
    "chars": 1325,
    "preview": "#!/bin/bash \n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  Se"
  },
  {
    "path": "conf/defaults.yaml",
    "chars": 5342,
    "preview": "# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See the NOTICE f"
  },
  {
    "path": "conf/jaas_digest.conf",
    "chars": 1353,
    "preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NOT"
  },
  {
    "path": "conf/logback.xml",
    "chars": 1141,
    "preview": "<?xml version=\"1.0\"?>\n<!--\n Licensed to the Apache Software Foundation (ASF) under one or more\n contributor license agre"
  },
  {
    "path": "conf/storm.yaml.example",
    "chars": 1614,
    "preview": "# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See the NOTICE f"
  },
  {
    "path": "logback/cluster.xml",
    "chars": 3064,
    "preview": "<?xml version=\"1.0\"?>\n<!--\n Licensed to the Apache Software Foundation (ASF) under one or more\n contributor license agre"
  },
  {
    "path": "project.clj",
    "chars": 1523,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-console-logging/project.clj",
    "chars": 1138,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/project.clj",
    "chars": 2853,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/LocalCluster.clj",
    "chars": 2508,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/LocalDRPC.clj",
    "chars": 1802,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/bootstrap.clj",
    "chars": 3697,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/clojure.clj",
    "chars": 7629,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/cluster.clj",
    "chars": 18258,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/activate.clj",
    "chars": 1031,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/config_value.clj",
    "chars": 994,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/deactivate.clj",
    "chars": 1037,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/dev_zookeeper.clj",
    "chars": 1119,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/kill_topology.clj",
    "chars": 1333,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/list.clj",
    "chars": 1990,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/rebalance.clj",
    "chars": 1888,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/command/shell_submission.clj",
    "chars": 1287,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/config.clj",
    "chars": 7029,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/acker.clj",
    "chars": 4537,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj",
    "chars": 4397,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/common.clj",
    "chars": 15262,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/drpc.clj",
    "chars": 6998,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/executor.clj",
    "chars": 41537,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/logviewer.clj",
    "chars": 4413,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/nimbus.clj",
    "chars": 57898,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/supervisor.clj",
    "chars": 21980,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/task.clj",
    "chars": 8758,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/daemon/worker.clj",
    "chars": 20895,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/disruptor.clj",
    "chars": 3637,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/event.clj",
    "chars": 2498,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/log.clj",
    "chars": 1300,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/messaging/loader.clj",
    "chars": 3186,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/messaging/local.clj",
    "chars": 2439,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/messaging/zmq.clj",
    "chars": 3763,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/metric/testing.clj",
    "chars": 2294,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/process_simulator.clj",
    "chars": 1566,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/scheduler/DefaultScheduler.clj",
    "chars": 3806,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/scheduler/EvenScheduler.clj",
    "chars": 4211,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/scheduler/IsolationScheduler.clj",
    "chars": 9452,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/stats.clj",
    "chars": 11684,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/testing.clj",
    "chars": 24601,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/testing4j.clj",
    "chars": 7377,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/thrift.clj",
    "chars": 8871,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/timer.clj",
    "chars": 4809,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/tuple.clj",
    "chars": 931,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/ui/core.clj",
    "chars": 32122,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/ui/helpers.clj",
    "chars": 4227,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/util.clj",
    "chars": 24952,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/backtype/storm/zookeeper.clj",
    "chars": 7634,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/storm/trident/testing.clj",
    "chars": 2593,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/clj/zilch/mq.clj",
    "chars": 3167,
    "preview": ";; Licensed to the Apache Software Foundation (ASF) under one\n;; or more contributor license agreements.  See the NOTICE"
  },
  {
    "path": "storm-core/src/dev/resources/tester_bolt.py",
    "chars": 1272,
    "preview": "# -*- coding: utf-8 -*-\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agre"
  },
  {
    "path": "storm-core/src/dev/resources/tester_bolt.rb",
    "chars": 1181,
    "preview": "# -*- coding: utf-8 -*-\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agre"
  },
  {
    "path": "storm-core/src/dev/resources/tester_spout.py",
    "chars": 1522,
    "preview": "# -*- coding: utf-8 -*-\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agre"
  },
  {
    "path": "storm-core/src/dev/resources/tester_spout.rb",
    "chars": 1474,
    "preview": "# -*- coding: utf-8 -*-\n\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agre"
  },
  {
    "path": "storm-core/src/genthrift.sh",
    "chars": 1026,
    "preview": "# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See the NOTICE f"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/Config.java",
    "chars": 43629,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/ConfigValidation.java",
    "chars": 3819,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/Constants.java",
    "chars": 1515,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/ILocalCluster.java",
    "chars": 2218,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/ILocalDRPC.java",
    "chars": 1126,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/StormSubmitter.java",
    "chars": 7312,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/clojure/ClojureBolt.java",
    "chars": 4037,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/clojure/ClojureSpout.java",
    "chars": 4491,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/clojure/RichShellBolt.java",
    "chars": 1900,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/clojure/RichShellSpout.java",
    "chars": 1879,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java",
    "chars": 3586,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/coordination/BatchOutputCollector.java",
    "chars": 1743,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/coordination/BatchOutputCollectorImpl.java",
    "chars": 1708,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/coordination/BatchSubtopologyBuilder.java",
    "chars": 15562,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/coordination/CoordinatedBolt.java",
    "chars": 14223,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/coordination/IBatchBolt.java",
    "chars": 1229,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/daemon/Shutdownable.java",
    "chars": 896,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java",
    "chars": 2818,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java",
    "chars": 6297,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/JoinResult.java",
    "chars": 2799,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/KeyedFairBolt.java",
    "chars": 3000,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/LinearDRPCInputDeclarer.java",
    "chars": 2179,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/LinearDRPCTopologyBuilder.java",
    "chars": 14700,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/PrepareRequest.java",
    "chars": 2300,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java",
    "chars": 3447,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java",
    "chars": 9846,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/Bolt.java",
    "chars": 13279,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/BoltStats.java",
    "chars": 43315,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java",
    "chars": 20710,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java",
    "chars": 24028,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ComponentObject.java",
    "chars": 13953,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java",
    "chars": 9864,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java",
    "chars": 13294,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java",
    "chars": 32368,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java",
    "chars": 64910,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java",
    "chars": 13602,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java",
    "chars": 13501,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ExecutorSpecificStats.java",
    "chars": 11128,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java",
    "chars": 24229,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java",
    "chars": 26054,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java",
    "chars": 13368,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/Grouping.java",
    "chars": 24276,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java",
    "chars": 9900,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/JavaObject.java",
    "chars": 15109,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/JavaObjectArg.java",
    "chars": 19042,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/KillOptions.java",
    "chars": 10202,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/Nimbus.java",
    "chars": 433375,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/NotAliveException.java",
    "chars": 9774,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/NullStruct.java",
    "chars": 6807,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java",
    "chars": 18807,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/ShellComponent.java",
    "chars": 13274,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java",
    "chars": 13436,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/SpoutStats.java",
    "chars": 27864,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java",
    "chars": 13928,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/StormTopology.java",
    "chars": 22192,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/StreamInfo.java",
    "chars": 14989,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java",
    "chars": 10709,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java",
    "chars": 23763,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java",
    "chars": 29842,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/TopologyInitialStatus.java",
    "chars": 1729,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/generated/TopologySummary.java",
    "chars": 28947,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/grouping/CustomStreamGrouping.java",
    "chars": 1752,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/BaseTaskHook.java",
    "chars": 1767,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/ITaskHook.java",
    "chars": 1512,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/info/BoltAckInfo.java",
    "chars": 1230,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/info/BoltExecuteInfo.java",
    "chars": 1250,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/info/BoltFailInfo.java",
    "chars": 1224,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/info/EmitInfo.java",
    "chars": 1275,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/info/SpoutAckInfo.java",
    "chars": 1214,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/hooks/info/SpoutFailInfo.java",
    "chars": 1200,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/messaging/IConnection.java",
    "chars": 1282,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/messaging/IContext.java",
    "chars": 2107,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/messaging/TaskMessage.java",
    "chars": 1577,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/messaging/TransportFactory.java",
    "chars": 2406,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/LoggingMetricsConsumer.java",
    "chars": 2617,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/MetricsConsumerBolt.java",
    "chars": 2485,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/SystemBolt.java",
    "chars": 5803,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/AssignableMetric.java",
    "chars": 1127,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/CombinedMetric.java",
    "chars": 1315,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java",
    "chars": 1233,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/ICombiner.java",
    "chars": 930,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/IMetric.java",
    "chars": 906,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/IMetricsConsumer.java",
    "chars": 2329,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/IReducer.java",
    "chars": 970,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/IStatefulObject.java",
    "chars": 899,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java",
    "chars": 1940,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java",
    "chars": 1525,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java",
    "chars": 1692,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/ReducedMetric.java",
    "chars": 1351,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/metric/api/StateMetric.java",
    "chars": 1086,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/nimbus/DefaultTopologyValidator.java",
    "chars": 1249,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/nimbus/ITopologyValidator.java",
    "chars": 1162,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/planner/CompoundSpout.java",
    "chars": 896,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/planner/CompoundTask.java",
    "chars": 894,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/planner/TaskBundle.java",
    "chars": 1138,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/Cluster.java",
    "chars": 14284,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/ExecutorDetails.java",
    "chars": 1686,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/INimbus.java",
    "chars": 1738,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/IScheduler.java",
    "chars": 1812,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/ISupervisor.java",
    "chars": 1775,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignment.java",
    "chars": 1673,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignmentImpl.java",
    "chars": 3488,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java",
    "chars": 2358,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/Topologies.java",
    "chars": 2056,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java",
    "chars": 2954,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java",
    "chars": 1553,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java",
    "chars": 4108,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/IAuthorizer.java",
    "chars": 1858,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java",
    "chars": 2021,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java",
    "chars": 3154,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java",
    "chars": 6766,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java",
    "chars": 4949,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java",
    "chars": 2967,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java",
    "chars": 2745,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java",
    "chars": 2228,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java",
    "chars": 2264,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/digest/ClientCallbackHandler.java",
    "chars": 4557,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java",
    "chars": 2889,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/auth/digest/ServerCallbackHandler.java",
    "chars": 5114,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/security/serialization/BlowfishTupleSerializer.java",
    "chars": 3751,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/serialization/DefaultKryoFactory.java",
    "chars": 2111,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/serialization/IKryoDecorator.java",
    "chars": 944,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/serialization/IKryoFactory.java",
    "chars": 1599,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/serialization/ITupleDeserializer.java",
    "chars": 989,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  },
  {
    "path": "storm-core/src/jvm/backtype/storm/serialization/ITupleSerializer.java",
    "chars": 983,
    "preview": "/**\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NO"
  }
]

// ... and 385 more files (download for full content)

About this extraction

This page contains the full source code of the nathanmarz/storm GitHub repository, extracted and formatted as plain text for AI agents and large language models (LLMs). The extraction includes 585 files (3.2 MB), approximately 885.6k tokens, and a symbol index with 7237 extracted functions, classes, methods, constants, and types. Use this with OpenClaw, Claude, ChatGPT, Cursor, Windsurf, or any other AI tool that accepts text input. You can copy the full output to your clipboard or download it as a .txt file.

Extracted by GitExtract — free GitHub repo to text converter for AI. Built by Nikandr Surkov.

Copied to clipboard!